You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by db...@apache.org on 2018/01/26 23:19:31 UTC

[1/3] trafodion git commit: [TRAFODION-2840] Make [first n] with ORDER BY views non-updatable

Repository: trafodion
Updated Branches:
  refs/heads/master c83471122 -> 2cdcdad51


[TRAFODION-2840] Make [first n] with ORDER BY views non-updatable


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

Branch: refs/heads/master
Commit: ebf7283de994729f898fa5a9f5e476fa03b40a4f
Parents: 63e1083
Author: Dave Birdsall <db...@apache.org>
Authored: Thu Jan 25 00:16:08 2018 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Thu Jan 25 00:16:08 2018 +0000

----------------------------------------------------------------------
 core/sql/generator/GenPreCode.cpp     | 18 +++++-
 core/sql/optimizer/BindRelExpr.cpp    | 35 +++++++++---
 core/sql/optimizer/Inlining.cpp       |  6 +-
 core/sql/optimizer/NormRelExpr.cpp    | 36 +++++++++++-
 core/sql/optimizer/OptPhysRelExpr.cpp | 89 ++++++++++++++++++++++++++++++
 core/sql/optimizer/RelExpr.cpp        |  4 +-
 core/sql/optimizer/RelMisc.h          | 19 ++++++-
 7 files changed, 192 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/generator/GenPreCode.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenPreCode.cpp b/core/sql/generator/GenPreCode.cpp
index cc17512..208ca0b 100644
--- a/core/sql/generator/GenPreCode.cpp
+++ b/core/sql/generator/GenPreCode.cpp
@@ -1604,8 +1604,20 @@ RelExpr * RelRoot::preCodeGen(Generator * generator,
   // can teach optimizer to do this.
   if ((getFirstNRows() != -1) || (getFirstNRowsParam()))
     {
+      // As of JIRA TRAFODION-2840, the binder always adds the FirstN,
+      // except in the case of output rowsets. So, the only time we 
+      // should now be going through this code is a SELECT query using
+      // output rowsets + FirstN + ORDER BY. A follow-on JIRA,
+      // TRAFODION-2924, will take care of that case and delete this code.
+      // (As a matter of design, it is highly undesireable to sometimes
+      // create the FirstN in the Binder and sometimes in the Generator;
+      // that means that any FirstN-related semantic checks in the 
+      // intervening passes will need two completely separate 
+      // implementations.)
+
       RelExpr * firstn = new(generator->wHeap()) FirstN(child(0),
 							getFirstNRows(),
+							needFirstSortedRows(),   
                                                         getFirstNRowsParam());
 
       // move my child's attributes to the firstN node.
@@ -5872,7 +5884,8 @@ RelExpr * GroupByAgg::preCodeGen(Generator * generator,
       && (getFirstNRows() == 1))
     {
       RelExpr * firstnNode = new(generator->wHeap()) FirstN(child(0),
-							    getFirstNRows());
+							    getFirstNRows(),
+							    FALSE /* [any n] is good enough */);
       firstnNode->setEstRowsUsed(getEstRowsUsed());
       firstnNode->setMaxCardEst(getMaxCardEst());
       firstnNode->setInputCardinality(child(0)->getInputCardinality());
@@ -10565,7 +10578,8 @@ RelExpr* PhyPack::preCodeGen(Generator* generator,
   if (getFirstNRows() != -1)
     {
       RelExpr * firstn = new(generator->wHeap()) FirstN(child(0),
-                                                        getFirstNRows());
+                                                        getFirstNRows(),
+                                                        FALSE /* [any n] is good enough */);
 
       // move my child's attributes to the firstN node.
       // Estimated rows will be mine.

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index 53abc71..7beaf9e 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -6662,17 +6662,38 @@ RelExpr *RelRoot::bindNode(BindWA *bindWA)
         }
     }
 
-  if ((NOT hasOrderBy()) &&
-      ((getFirstNRows() != -1) ||
-       (getFirstNRowsParam())))
+  if ((getFirstNRows() != -1) ||
+       (getFirstNRowsParam()))
     {
       // create a firstN node to retrieve firstN rows.
       FirstN * firstn = new(bindWA->wHeap())
-        FirstN(child(0), getFirstNRows(), getFirstNRowsParam());
+        FirstN(child(0), getFirstNRows(), needFirstSortedRows(), getFirstNRowsParam());   
+
       firstn->bindNode(bindWA);
       if (bindWA->errStatus())
         return NULL;
 
+      // Note: For ORDER BY + [first n], we want to sort the rows before 
+      // picking just n of them. (We don't do this for [any n].) We might
+      // be tempted to copy the orderByTree into the FirstN node at this
+      // point, but this doesn't work. Instead, we copy the bound ValueIds
+      // at normalize time. We have to do this in case there are expressions
+      // involved in the ORDER BY and there is a DESC. The presence of the
+      // Inverse node at the top of the expression tree seems to cause the
+      // expressions underneath to be bound to different ValueIds, which 
+      // causes coverage tests in FirstN::createContextForAChild requirements
+      // generation to fail. An example of where this occurs is:
+      //
+      // prepare s1 from
+      //   select [first 2] y, x from
+      //    (select a,b + 26 from t1) as t(x,y)
+      //   order by y desc;
+      //
+      // If we copy the ORDER BY ItemExpr tree and rebind, we get a different
+      // ValueId for the expression b + 26 in the child characteristic outputs
+      // than what we get for the child of Inverse in Inverse(B + 26). The
+      // trick of copying the already-bound ORDER BY clause later avoids this.
+
       setChild(0, firstn);
 
       // reset firstN indication in the root node.
@@ -11545,7 +11566,7 @@ RelExpr *Update::bindNode(BindWA *bindWA)
       if (scanNode->getFirstNRows() >= 0)
         {
           FirstN * firstn = new(bindWA->wHeap())
-            FirstN(scanNode, scanNode->getFirstNRows(), NULL);
+            FirstN(scanNode, scanNode->getFirstNRows(), FALSE /* there's no ORDER BY on an UPDATE */, NULL);
           firstn->bindNode(bindWA);
           if (bindWA->errStatus())
             return NULL;
@@ -11901,7 +11922,7 @@ RelExpr *Delete::bindNode(BindWA *bindWA)
 
       RelExpr * childNode = child(0)->castToRelExpr();
       FirstN * firstn = new(bindWA->wHeap())
-        FirstN(childNode, getFirstNRows(), NULL);
+        FirstN(childNode, getFirstNRows(), FALSE /* There's no ORDER BY on a DELETE */, NULL);
       firstn->bindNode(bindWA);
       if (bindWA->errStatus())
         return NULL;
@@ -11988,7 +12009,7 @@ RelExpr *Delete::bindNode(BindWA *bindWA)
       // during handleInlining. Occurs when DELETE FIRST N is used on table with no
       // dependent objects. 
       FirstN * firstn = new(bindWA->wHeap())
-        FirstN(boundExpr, getFirstNRows());
+        FirstN(boundExpr, getFirstNRows(), FALSE /* There's no ORDER BY on a DELETE */ );
       if (NOT(scanNode && scanNode->getSelectionPred().containsSubquery()))
         firstn->setCanExecuteInDp2(TRUE);
       firstn->bindNode(bindWA);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/Inlining.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/Inlining.cpp b/core/sql/optimizer/Inlining.cpp
index a3c096a..f8d7b66 100644
--- a/core/sql/optimizer/Inlining.cpp
+++ b/core/sql/optimizer/Inlining.cpp
@@ -3132,7 +3132,7 @@ RelExpr *GenericUpdate::inlineOnlyRIandIMandMVLogging(BindWA *bindWA,
 	{
 	  // create a firstN node to delete N rows.
 	  FirstN * firstn = new(bindWA->wHeap())
-	    FirstN(topNode, topNode->getFirstNRows());
+	    FirstN(topNode, topNode->getFirstNRows(), FALSE /* No ordering requirement */);
 	  firstn->bindNode(bindWA);
 	  if (bindWA->errStatus())
 	    return NULL;
@@ -3314,7 +3314,7 @@ RelExpr *GenericUpdate::inlineAfterOnlyBackbone(BindWA *bindWA,
   {
     // create a firstN node to delete N rows.
     FirstN * firstn = new(bindWA->wHeap())
-      FirstN(topNode, topNode->getFirstNRows());
+      FirstN(topNode, topNode->getFirstNRows(), FALSE /* No ordering requirement */);
     firstn->bindNode(bindWA);
     if (bindWA->errStatus())
       return NULL;
@@ -3429,7 +3429,7 @@ RelExpr *GenericUpdate::inlineAfterOnlyBackboneForUndo(BindWA *bindWA,
     {
       // create a firstN node to delete N rows.
       FirstN * firstn = new(bindWA->wHeap())
-	FirstN(topNode, topNode->getFirstNRows());
+	FirstN(topNode, topNode->getFirstNRows(), FALSE /* No ordering requirement */);
       firstn->bindNode(bindWA);
       if (bindWA->errStatus())
 	return NULL;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/NormRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NormRelExpr.cpp b/core/sql/optimizer/NormRelExpr.cpp
index 781ecc9..95c526b 100644
--- a/core/sql/optimizer/NormRelExpr.cpp
+++ b/core/sql/optimizer/NormRelExpr.cpp
@@ -6925,6 +6925,22 @@ RelExpr * Insert::normalizeNode(NormWA & normWARef)
   if (normalizedThis->getOperatorType() == REL_LEAF_INSERT)
     return normalizedThis;
 
+  // If there is an ORDER BY + a [first n], copy the ORDER BY ValueIds
+  // down to the FirstN node so we order the rows before taking the first n.
+  // If it is ORDER BY + [any n] we don't do this, as it is sufficient
+  // and more efficient to sort the rows after taking just n of them.
+  // Note: We do this at normalize time instead of bind time because if
+  // there are complex expressions in the ORDER BY, the binder will get
+  // different ValueIds for the non-leaf nodes which screws up coverage
+  // tests. Doing it here the ValueIds have already been uniquely computed.
+  if ((reqdOrder().entries() > 0) && 
+      (child(0)->getOperatorType() == REL_FIRST_N))
+    {
+      FirstN * firstn = (FirstN *)child(0)->castToRelExpr();
+      if (firstn->isFirstN())  // that is, [first n], not [any n] or [last n]
+        firstn->reqdOrder().insert(reqdOrder());
+    }
+
   // If the child is not a Tuple node - nothing to do here.
   CMPASSERT(normalizedThis->getArity() > 0);
   if (normalizedThis->child(0)->getOperatorType() != REL_TUPLE)
@@ -7597,6 +7613,24 @@ RelExpr * RelRoot::normalizeNode(NormWA & normWARef)
   }
 
   // ---------------------------------------------------------------------
+  // If there is an ORDER BY + a [first n], copy the ORDER BY ValueIds
+  // down to the FirstN node so we order the rows before taking the first n.
+  // If it is ORDER BY + [any n] we don't do this, as it is sufficient
+  // and more efficient to sort the rows after taking just n of them.
+  // Note: We do this at normalize time instead of bind time because if
+  // there are complex expressions in the ORDER BY, the binder will get
+  // different ValueIds for the non-leaf nodes which screws up coverage
+  // tests. Doing it here the ValueIds have already been uniquely computed.
+  // ---------------------------------------------------------------------
+  if ((reqdOrder().entries() > 0) && 
+      (child(0)->getOperatorType() == REL_FIRST_N))
+    {
+      FirstN * firstn = (FirstN *)child(0)->castToRelExpr();
+      if (firstn->isFirstN())  // that is, [first n], not [any n] or [last n]
+        firstn->reqdOrder().insert(reqdOrder());
+    }
+
+  // ---------------------------------------------------------------------
   // Normalize the child.
   // ---------------------------------------------------------------------
   child(0) = child(0)->normalizeNode(normWARef);
@@ -10908,5 +10942,5 @@ NABoolean CqsWA::isMPTable(const NAString &tableName)
     return FALSE;
   }
 } // CqsWA::isMPTable()
- 
+
  

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/OptPhysRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/OptPhysRelExpr.cpp b/core/sql/optimizer/OptPhysRelExpr.cpp
index 838eae4..eff9944 100644
--- a/core/sql/optimizer/OptPhysRelExpr.cpp
+++ b/core/sql/optimizer/OptPhysRelExpr.cpp
@@ -15499,6 +15499,95 @@ GenericUtilExpr::synthPhysicalProperty(const Context* myContext,
   return sppForMe;
 } //  GenericUtilExpr::synthPhysicalProperty()
 
+// -----------------------------------------------------------------------
+// FirstN::createContextForAChild()
+// The FirstN node may have an order by requirement that it needs to
+// pass to its child context. Other than that, this method is quite
+// similar to the default implementation, RelExpr::createContextForAChild.
+// The arity of FirstN is always 1, so some logic from the default
+// implementation that deals with childIndex > 0 is unnecessary and has
+// been removed.
+// -----------------------------------------------------------------------
+Context * FirstN::createContextForAChild(Context* myContext,
+                                 PlanWorkSpace* pws,
+                                 Lng32& childIndex)
+{
+  const ReqdPhysicalProperty* rppForMe =
+                                    myContext->getReqdPhysicalProperty();
+
+  CMPASSERT(getArity() == 1);
+
+  childIndex = getArity() - pws->getCountOfChildContexts() - 1;
+
+  // return if we are done
+  if (childIndex < 0)
+    return NULL;
+
+  RequirementGenerator rg(child(childIndex), rppForMe);
+
+  if (reqdOrder().entries() > 0)
+    {
+      // replace our sort requirement with that implied by our ORDER BY clause
+
+      rg.removeSortKey();
+
+      ValueIdList sortKey;
+      sortKey.insert(reqdOrder());
+
+      // Shouldn't/Can't add a sort order type requirement
+      // if we are in DP2
+      if (rppForMe->executeInDP2())
+        rg.addSortKey(sortKey,NO_SOT);
+      else
+        rg.addSortKey(sortKey,ESP_SOT);
+    }
+
+  if (NOT pws->isEmpty())
+  {
+    const Context* childContext = pws->getLatestChildContext();
+
+    // ------------------------------------------------------------------
+    // Cost limit exceeded or got no solution? Give up since we only
+    // try one plan.
+    // ------------------------------------------------------------------
+    if(NOT (childContext AND childContext->hasOptimalSolution()))
+      return NULL;
+
+    if (NOT pws->isLatestContextWithinCostLimit())
+      return NULL;
+
+  }
+
+  if (NOT rg.checkFeasibility())
+    return NULL;
+
+  Lng32 planNumber = 0;
+
+  // ---------------------------------------------------------------------
+  // Compute the cost limit to be applied to the child.
+  // ---------------------------------------------------------------------
+  CostLimit* costLimit = computeCostLimit(myContext, pws);
+
+  // ---------------------------------------------------------------------
+  // Get a Context for optimizing the child.
+  // Search for an existing Context in the CascadesGroup to which the
+  // child belongs that requires the same properties as myContext.
+  // Reuse it, if found. Otherwise, create a new Context that contains
+  // the same rpp and input log prop as in myContext.
+  // ---------------------------------------------------------------------
+  Context* result = shareContext(childIndex, rg.produceRequirement(),
+                                 myContext->getInputPhysicalProperty(),
+                                 costLimit, myContext,
+                                 myContext->getInputLogProp());
+
+  // ---------------------------------------------------------------------
+  // Store the Context for the child in the PlanWorkSpace.
+  // ---------------------------------------------------------------------
+  pws->storeChildContext(childIndex, planNumber, result);
+
+  return result;
+
+} // FirstN::createContextForAChild()
 
 //<pb>
 //==============================================================================

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/RelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExpr.cpp b/core/sql/optimizer/RelExpr.cpp
index c9d3f83..5c0af91 100644
--- a/core/sql/optimizer/RelExpr.cpp
+++ b/core/sql/optimizer/RelExpr.cpp
@@ -11651,13 +11651,15 @@ RelExpr * FirstN::copyTopNode(RelExpr *derivedNode,
   FirstN *result;
 
   if (derivedNode == NULL) {
-    result = new (outHeap) FirstN(NULL, getFirstNRows(), getFirstNRowsParam(),
+    result = new (outHeap) FirstN(NULL, getFirstNRows(), isFirstN(), getFirstNRowsParam(),
                                   outHeap);
     result->setCanExecuteInDp2(canExecuteInDp2());
   }
   else
     result = (FirstN *) derivedNode;
 
+  result->reqdOrder().insert(reqdOrder());
+
   return RelExpr::copyTopNode(result, outHeap);
 }
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/ebf7283d/core/sql/optimizer/RelMisc.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelMisc.h b/core/sql/optimizer/RelMisc.h
index d25d21a..e6920ca 100644
--- a/core/sql/optimizer/RelMisc.h
+++ b/core/sql/optimizer/RelMisc.h
@@ -1600,12 +1600,14 @@ class FirstN : public RelExpr
 public:
  FirstN(RelExpr * child,
         Int64 firstNRows,
+        NABoolean isFirstN,
         ItemExpr * firstNRowsParam = NULL,
         CollHeap *oHeap = CmpCommon::statementHeap())
    : RelExpr(REL_FIRST_N, child, NULL, oHeap),
     firstNRows_(firstNRows),
     firstNRowsParam_(firstNRowsParam),
-    canExecuteInDp2_(FALSE)
+    canExecuteInDp2_(FALSE),
+    isFirstN_(isFirstN)
     {
       setNonCacheable();
     };
@@ -1613,6 +1615,12 @@ public:
   // sets the canExecuteInDp2 flag for the [LAST 1] operator
   // of an MTS delete and calls the base class implementation of bindNode.
   virtual RelExpr* bindNode(BindWA* bindWA);
+
+  // takes care of any ordering requirement on the child
+  virtual Context* createContextForAChild(Context* myContext,
+                     PlanWorkSpace* pws,
+                     Lng32& childIndex);
+
   //
   // Physical properties implemented in OptPhysRelExpr.cpp
   //
@@ -1650,11 +1658,20 @@ public:
   NABoolean canExecuteInDp2() const             { return canExecuteInDp2_; }
   virtual NABoolean computeRowsAffected()   const ;
 
+  NABoolean isFirstN()                          { return isFirstN_; }
+
+  ValueIdList & reqdOrder()                     { return reqdOrder_; }
+
 private:
   // Otherwise, return firstNRows_ at runtime.
   Int64 firstNRows_;
   ItemExpr * firstNRowsParam_;
   NABoolean canExecuteInDp2_;
+  NABoolean isFirstN_;  // TRUE if [first n], FALSE if [any n] or [last n]
+
+  // Optional ORDER BY to force ordering before applying First N; populated
+  // at normalizeNode time.
+  ValueIdList reqdOrder_;
 
 }; // class FirstN
 


[2/3] trafodion git commit: Reworks as suggested by Hans

Posted by db...@apache.org.
Reworks as suggested by Hans


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

Branch: refs/heads/master
Commit: 97f137c5cdd4d9a7568a063f03a369a914eac839
Parents: ebf7283
Author: Dave Birdsall <db...@apache.org>
Authored: Thu Jan 25 19:35:07 2018 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Thu Jan 25 19:35:07 2018 +0000

----------------------------------------------------------------------
 core/sql/optimizer/OptPhysRelExpr.cpp | 27 +++------------------------
 1 file changed, 3 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/97f137c5/core/sql/optimizer/OptPhysRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/OptPhysRelExpr.cpp b/core/sql/optimizer/OptPhysRelExpr.cpp
index eff9944..1f7a4dc 100644
--- a/core/sql/optimizer/OptPhysRelExpr.cpp
+++ b/core/sql/optimizer/OptPhysRelExpr.cpp
@@ -15527,37 +15527,16 @@ Context * FirstN::createContextForAChild(Context* myContext,
 
   if (reqdOrder().entries() > 0)
     {
-      // replace our sort requirement with that implied by our ORDER BY clause
-
-      rg.removeSortKey();
-
-      ValueIdList sortKey;
-      sortKey.insert(reqdOrder());
+      // add our sort requirement as implied by our ORDER BY clause
 
       // Shouldn't/Can't add a sort order type requirement
       // if we are in DP2
       if (rppForMe->executeInDP2())
-        rg.addSortKey(sortKey,NO_SOT);
+        rg.addSortKey(reqdOrder(),NO_SOT);
       else
-        rg.addSortKey(sortKey,ESP_SOT);
+        rg.addSortKey(reqdOrder(),ESP_SOT);
     }
 
-  if (NOT pws->isEmpty())
-  {
-    const Context* childContext = pws->getLatestChildContext();
-
-    // ------------------------------------------------------------------
-    // Cost limit exceeded or got no solution? Give up since we only
-    // try one plan.
-    // ------------------------------------------------------------------
-    if(NOT (childContext AND childContext->hasOptimalSolution()))
-      return NULL;
-
-    if (NOT pws->isLatestContextWithinCostLimit())
-      return NULL;
-
-  }
-
   if (NOT rg.checkFeasibility())
     return NULL;
 


[3/3] trafodion git commit: Merge [TRAFODION-2840] PR 1414 Make [first n} + ORDER BY views non-updatable

Posted by db...@apache.org.
Merge [TRAFODION-2840] PR 1414 Make [first n} + ORDER BY views non-updatable


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

Branch: refs/heads/master
Commit: 2cdcdad51c43f2f92b05583ca5dc6ac31dd9a06f
Parents: c834711 97f137c
Author: Dave Birdsall <db...@apache.org>
Authored: Fri Jan 26 23:18:24 2018 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Fri Jan 26 23:18:24 2018 +0000

----------------------------------------------------------------------
 core/sql/generator/GenPreCode.cpp     | 18 +++++++-
 core/sql/optimizer/BindRelExpr.cpp    | 35 ++++++++++++---
 core/sql/optimizer/Inlining.cpp       |  6 +--
 core/sql/optimizer/NormRelExpr.cpp    | 36 +++++++++++++++-
 core/sql/optimizer/OptPhysRelExpr.cpp | 68 ++++++++++++++++++++++++++++++
 core/sql/optimizer/RelExpr.cpp        |  4 +-
 core/sql/optimizer/RelMisc.h          | 19 ++++++++-
 7 files changed, 171 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/2cdcdad5/core/sql/optimizer/OptPhysRelExpr.cpp
----------------------------------------------------------------------