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 2017/10/05 15:25:10 UTC

[1/5] incubator-trafodion git commit: [TRAFODION-1610][TRAFODION-2630] Generate constr exprs on update operators

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 4706888f1 -> f9ba966ed


[TRAFODION-1610][TRAFODION-2630] Generate constr exprs on update operators


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

Branch: refs/heads/master
Commit: c22b6d10fdd3065a948163d64a20c3039128474c
Parents: fc575b1
Author: Dave Birdsall <db...@apache.org>
Authored: Tue Oct 3 16:09:29 2017 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Tue Oct 3 16:09:29 2017 +0000

----------------------------------------------------------------------
 core/sql/generator/GenRelUpdate.cpp      | 203 ++++++++++++++++----------
 core/sql/optimizer/BindRelExpr.cpp       |  28 ----
 core/sql/optimizer/Inlining.cpp          | 104 -------------
 core/sql/optimizer/RelUpdate.h           |   4 -
 core/sql/regress/executor/EXPECTED015.SB | 150 +++++++++++++++++++
 core/sql/regress/executor/TEST015        |  62 ++++++++
 6 files changed, 335 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/generator/GenRelUpdate.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelUpdate.cpp b/core/sql/generator/GenRelUpdate.cpp
index a744bed..9b9b4e7 100644
--- a/core/sql/generator/GenRelUpdate.cpp
+++ b/core/sql/generator/GenRelUpdate.cpp
@@ -292,8 +292,85 @@ static short genUpdExpr(
   return 0;
 }
 
-static short genUpdConstraintExpr(Generator *generator)
+// Used to generate update or insert constraint expressions for update operators
+static short genUpdConstraintExpr(Generator * generator,
+                                  ItemExpr * constrTree,
+                                  const ValueIdSet & constraintColumns,
+                                  ValueIdArray & targetRecExprArray,
+                                  ex_expr ** targetExpr /* out */)
 {
+  ExpGenerator * expGen = generator->getExpGenerator();
+
+  // The Attributes for the table columns refer to the old values of the column.
+  // The constraints must operate on the new values, though. So we must do a
+  // switcheroo on the Attributes for the update expression. The target value IDs
+  // come from targetRecExprArray.
+
+  ValueIdList savedSourceVIDlist;
+  NAList<Attributes*> savedSourceAttrsList(generator->wHeap());
+
+  for (ValueId sourceValId = constraintColumns.init();
+       constraintColumns.next(sourceValId);
+       constraintColumns.advance(sourceValId))
+    {
+      NAColumn * sourceCol = ((IndexColumn*)sourceValId.getItemExpr())->getNAColumn();
+      ValueId targetValId;
+      NAColumn *targetCol = NULL;
+      NABoolean found = FALSE;
+      for (CollIndex ni = 0; (!found) && (ni < targetRecExprArray.entries()); ni++)
+        {
+          const ItemExpr *assignExpr = targetRecExprArray[ni].getItemExpr();
+          targetValId = assignExpr->child(0)->castToItemExpr()->getValueId();            
+          if (targetValId.getItemExpr()->getOperatorType() == ITM_BASECOLUMN)
+            targetCol = ((BaseColumn*)targetValId.getItemExpr())->getNAColumn();
+          else if (targetValId.getItemExpr()->getOperatorType() == ITM_INDEXCOLUMN)
+            targetCol = ((IndexColumn*)targetValId.getItemExpr())->getNAColumn();
+                
+          if ((targetCol) &&
+              (targetCol->getColName() == sourceCol->getColName()) &&
+              (targetCol->getHbaseColFam() == sourceCol->getHbaseColFam()) &&
+              (targetCol->getHbaseColQual() == sourceCol->getHbaseColQual()) &&
+              (targetCol->getNATable()->getTableName().getQualifiedNameAsAnsiString() ==
+               sourceCol->getNATable()->getTableName().getQualifiedNameAsAnsiString()))
+            {
+              found = TRUE;
+              break;
+            }
+        }
+
+      if (found)
+	{
+          Attributes * sourceValAttr = (generator->addMapInfo(sourceValId, 0))->getAttr();
+          Attributes * targetValAttr = (generator->getMapInfo(targetValId, 0))->getAttr();
+
+          // Save original location attributes so we can change them back after
+          // generating the update constraint expression
+
+          Attributes * savedValAttr = new(generator->wHeap()) Attributes();
+          savedValAttr->copyLocationAttrs(sourceValAttr);
+          savedSourceAttrsList.insert(savedValAttr);
+          savedSourceVIDlist.insert(sourceValId);
+
+          sourceValAttr->copyLocationAttrs(targetValAttr);
+        }
+
+    }
+
+  // Now that we have remapped the Attributes for the columns to their values
+  // in the new record, we can generate the update constraint expression.
+
+  expGen->generateExpr(constrTree->getValueId(), ex_expr::exp_SCAN_PRED,
+                       targetExpr);
+
+  // Now put the Attributes back the way they were.
+
+  for (Lng32 i = 0; i < savedSourceVIDlist.entries(); i++)
+    {
+      ValueId sourceValId = savedSourceVIDlist[i];
+      Attributes * sourceValAttr = (generator->getMapInfo(sourceValId, 0))->getAttr();
+      sourceValAttr->copyLocationAttrs(savedSourceAttrsList[i]);
+    }
+
   return 0;
 }
 
@@ -1718,19 +1795,56 @@ short HbaseUpdate::codeGen(Generator * generator)
     }
   else if (getIndexDesc()->isClusteringIndex() && getCheckConstraints().entries())
     {
-      GenAssert(FALSE, "Should not reach here. This update should have been transformed to delete/insert");
-      // To be uncommented when TRAFODION-1610 is implemented
-      // Need to generate insConstraintExpr also
-/*
+      // Generate the update and insert constraint check expressions
+
+      // The attributes for the columns referenced in the constraint expressions
+      // refer to the source values of the columns. We want to evaluate the
+      // constraints aganst the target values, though. So, there is some
+      // Attributes gymnastics that has to happen to generate them.
+
+      // Obtain the ValueIds of base table columns referenced in the
+      // constraints
+
+      ValueId constraintId;
+      ValueIdSet constraintColumns;
+      for (CollIndex ci = 0; ci < getCheckConstraints().entries(); ci++)
+        {
+          constraintId = getCheckConstraints()[ci];
+          constraintId.getItemExpr()->findAll(ITM_INDEXCOLUMN,
+                                              constraintColumns, // out, has append semantics
+                                              TRUE, // visitVEGmembers
+                                              FALSE); // don't visit index descriptors
+        }
+
+      // Prepare the constraint tree for generation
+
       ItemExpr *constrTree =
         getCheckConstraints().rebuildExprTree(ITM_AND, TRUE, TRUE);
 
       if (getTableDesc()->getNATable()->hasSerializedEncodedColumn())
-        constrTree = generator->addCompDecodeForDerialization(constrTree);
+        constrTree = generator->addCompDecodeForDerialization(constrTree, isAlignedFormat);
+
+      // Generate the update constraint expression, substituting Attributes for
+      // the new update record
+
+      genUpdConstraintExpr(generator,
+                           constrTree,
+                           constraintColumns,
+                           newRecExprArray(),
+                           &updConstraintExpr /* out */);
+
+      if ((isMerge()) && (mergeInsertRecExprArray().entries() > 0))
+        {
+          // Generate the insert constraint expression, substituting Attributes for
+          // the new insert record
+
+          genUpdConstraintExpr(generator,
+                               constrTree,
+                               constraintColumns,
+                               mergeInsertRecExprArray(),
+                               &insConstraintExpr /* out */);   
+        }
 
-      expGen->generateExpr(constrTree->getValueId(), ex_expr::exp_SCAN_PRED,
-                            &updConstraintExpr);
-*/
     }
  
   if ((getTableDesc()->getNATable()->isSeabaseTable()) &&
@@ -2409,9 +2523,6 @@ short HbaseInsert::codeGen(Generator *generator)
   ex_expr * rowIdExpr = NULL;
   ULng32 rowIdLen = 0;
 
-  ValueIdList savedInputVIDlist;
-  NAList<Attributes*> savedInputAttrsList(generator->wHeap());
-
   const ValueIdList &indexVIDlist = getIndexDesc()->getIndexColumns();
   for (CollIndex ii = 0; ii < newRecExprArray().entries(); ii++)
     {
@@ -2427,65 +2538,6 @@ short HbaseInsert::codeGen(Generator *generator)
       
       colAttr->copyLocationAttrs(castAttr);
       indexAttr->copyLocationAttrs(castAttr);
-      // To be removed when TRAFODION-1610 is implemented
-      //  `
-      // if any of the target column is also an input value to this operator, then
-      // make the value id of that input point to the location of the target column.
-      // This is done as the input column value will become the target after this
-      // insert expr is evaluated.
-      // This is done if this value will be part of an expression that need to
-      // be evaluated on the updated columns.
-      const ValueIdSet& inputSet = getGroupAttr()->getCharacteristicInputs();
-      ValueId inputValId;
-      if ((inputSet.entries() > 0) &&
-          (getIndexDesc()->isClusteringIndex() && getCheckConstraints().entries()))
-	{
-	  NAColumn *inputCol = NULL;
-	  NABoolean found = FALSE;
-	  for (inputValId = inputSet.init();
-	       ((NOT found) && (inputSet.next(inputValId)));
-	       inputSet.advance(inputValId) )
-	    {
-	      if ((inputValId.getItemExpr()->getOperatorType() != ITM_BASECOLUMN) &&
-		  (inputValId.getItemExpr()->getOperatorType() != ITM_INDEXCOLUMN))
-		{
-		  continue;
-		}
-	      
-	      if (inputValId.getItemExpr()->getOperatorType() == ITM_BASECOLUMN)
-		{
-		  inputCol = ((BaseColumn*)inputValId.getItemExpr())->getNAColumn();
-		}
-	      else
-		{
-		  inputCol = ((IndexColumn*)inputValId.getItemExpr())->getNAColumn();
-		}
-	      
-	      if ((col->getColName() == inputCol->getColName()) &&
-                  (col->getHbaseColFam() == inputCol->getHbaseColFam()) &&
-                  (col->getHbaseColQual() == inputCol->getHbaseColQual()) &&
-                  (col->getNATable()->getTableName().getQualifiedNameAsAnsiString() ==
-                   inputCol->getNATable()->getTableName().getQualifiedNameAsAnsiString()))
-		{
-		  found = TRUE;
-		  break;
-		}
-	    } // for
-
-	  if (found)
-	    {
-	      Attributes * inputValAttr = (generator->addMapInfo(inputValId, 0))->getAttr();
-
-              // save original location attributes. These will be restored back once
-              // constr expr has been generated.
-              Attributes * savedValAttr = new(generator->wHeap()) Attributes();
-              savedValAttr->copyLocationAttrs(inputValAttr);
-              savedInputAttrsList.insert(savedValAttr);
-              savedInputVIDlist.insert(inputValId);
-
-	      inputValAttr->copyLocationAttrs(castAttr);
-	    }
-	} // if
     }
 
   ex_expr* preCondExpr = NULL;
@@ -2520,15 +2572,6 @@ short HbaseInsert::codeGen(Generator *generator)
 
       expGen->generateExpr(constrTree->getValueId(), ex_expr::exp_SCAN_PRED,
 			   &insConstraintExpr);
-
-      // restore original attribute values
-      // To be removed when TRAFODION-1610 is implemented
-      for (Lng32 i = 0; i < savedInputVIDlist.entries(); i++)
-        {
-          ValueId inputValId = savedInputVIDlist[i];
-          Attributes * inputValAttr = (generator->getMapInfo(inputValId, 0))->getAttr();
-          inputValAttr->copyLocationAttrs(savedInputAttrsList[i]);
-        }
     }
   
   listOfUpdatedColNames = new(space) Queue(space);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index 266e23f..f67b865 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -11552,35 +11552,7 @@ RelExpr *Update::bindNode(BindWA *bindWA)
   NABoolean transformUpdateKey = updatesClusteringKeyOrUniqueIndexKey(bindWA);
   if (bindWA->errStatus()) // error occurred in updatesCKOrUniqueIndexKey()
     return this;
-  // To be removed when TRAFODION-1610 is implemented.
-  NABoolean xnsfrmHbaseUpdate = FALSE;
-  if ((hbaseOper()) && (NOT isMerge()))
-    {      
-      if (CmpCommon::getDefault(HBASE_TRANSFORM_UPDATE_TO_DELETE_INSERT) == DF_ON)
-	{
-	  xnsfrmHbaseUpdate = TRUE;
-	}
-      else if (getCheckConstraints().entries())
-       {
-         xnsfrmHbaseUpdate = TRUE;
-       }
-      else if (getTableDesc()->getNATable()->isHbaseMapTable())
-       {
-         xnsfrmHbaseUpdate = TRUE;
-       }
-     }  
-  
-  if (xnsfrmHbaseUpdate)
-    {
-      ULng32 savedParserFlags = Get_SqlParser_Flags (0xFFFFFFFF);
-      Set_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL);
 
-      boundExpr = transformHbaseUpdate(bindWA);
-
-      Assign_SqlParser_Flags (savedParserFlags);
-    }
-  else 
-  // till here and remove the function transformHbaseUpdate also
   if ((transformUpdateKey) && (NOT isMerge()))
     {
       boundExpr = transformUpdatePrimaryKey(bindWA);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/optimizer/Inlining.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/Inlining.cpp b/core/sql/optimizer/Inlining.cpp
index f90a560..73b7f4a 100644
--- a/core/sql/optimizer/Inlining.cpp
+++ b/core/sql/optimizer/Inlining.cpp
@@ -4934,107 +4934,3 @@ RelExpr *Update::transformUpdatePrimaryKey(BindWA *bindWA)
    return boundExpr;
 }
 
-RelExpr *Update::transformHbaseUpdate(BindWA *bindWA)
-{
-
-   Delete * delNode = new (bindWA->wHeap())
-			Delete(CorrName(getTableDesc()->getCorrNameObj(), bindWA->wHeap()),
-				NULL,
-				REL_UNARY_DELETE,
-				child(0),
-				NULL);
-   delNode->setNoLogOp(CONSISTENT_NOLOG);
-   delNode->setUpdateCKorUniqueIndexKey(TRUE);
-   delNode->rowsAffected() = GenericUpdate::DO_NOT_COMPUTE_ROWSAFFECTED;
-
-   ValueIdList selectList, sourceColsList, lhsOfSetClause;
-   
-   getTableDesc()->getUserColumnList(selectList);
-   getScanIndexDesc()->getPrimaryTableDesc()->getUserColumnList(sourceColsList);
-   ValueId vid ;
-   CollIndex pos;
-
-   // newRecExprArray is a list of assigns. For each assign
-   // child(0) is the LHS of the set clause and child(1) is
-   // the RHS of the SET clause
-
-   for (CollIndex i=0; i < newRecExprArray().entries(); i++)
-    {
-      lhsOfSetClause.insertAt(i,newRecExprArray().at(i).getItemExpr()->child(0).getValueId());
-    }
-
-   for (CollIndex i=0; i < selectList.entries(); i++)
-    {
-      if ((pos = lhsOfSetClause.index(selectList[i])) == NULL_COLL_INDEX)
-	selectList[i] = sourceColsList[i];
-      else
-	selectList[i] = newRecExprArray().at(pos).getItemExpr()->child(1).getValueId();
-    }
-
-    for (CollIndex i=0; i < oldToNewMap().getTopValues().entries(); i++) {
-      BaseColumn *col = (BaseColumn *) oldToNewMap().getBottomValues()[i].getItemExpr();
-      NABoolean addToOldToNewMap = TRUE;
-
-      // Copy the oldToNewMap.
-      if (col->getNAColumn()->isComputedColumnAlways()) {
-        // Computed columns can be copied from delete to insert if they don't
-        // change. Don't include the column in this map, though, if one of
-        // the underlying columns gets updated, because the value of the
-        // computed column has to be recomputed. That computation will be
-        // done in the new insert node.
-        ValueIdSet underlyingCols;
-
-        col->getUnderlyingColumnsForCC(underlyingCols);
-
-        if (NOT underlyingCols.intersect(lhsOfSetClause).isEmpty())
-          addToOldToNewMap = FALSE;
-      }
-
-      // Copy the oldToNewMap.
-      if (addToOldToNewMap)
-        delNode->oldToNewMap().addMapEntry(oldToNewMap().getTopValues()[i],
-                                           oldToNewMap().getBottomValues()[i]);
-    }
-
-   RelRoot * rootNode = new (bindWA->wHeap())
-			RelRoot(delNode, 
-				REL_ROOT,
-				selectList.rebuildExprTree(ITM_ITEM_LIST));
-
-   RelExpr * boundExpr;
-   Insert * insNode = new (bindWA->wHeap())
-		Insert(CorrName(getTableDesc()->getCorrNameObj(),bindWA->wHeap()),
-				getTableDesc(), // insert gets the same tabledesc as the update
-				REL_UNARY_INSERT,
-				rootNode,
-				NULL);
-   insNode->setNoLogOp(isNoLogOperation());
-   insNode->setSubqInUpdateAssign(subqInUpdateAssign());
-
-   if (this->rowsAffected() == GenericUpdate::DO_NOT_COMPUTE_ROWSAFFECTED)
-      insNode->rowsAffected() = GenericUpdate::DO_NOT_COMPUTE_ROWSAFFECTED;
-   else
-      insNode->rowsAffected() = GenericUpdate::COMPUTE_ROWSAFFECTED;
-      
-   insNode->setUpdateCKorUniqueIndexKey(TRUE);
-   InliningInfo inlineInfo = getInliningInfo();
-   insNode->setInliningInfo(&inlineInfo);
-   if (CmpCommon::getDefault(UPDATE_CLUSTERING_OR_UNIQUE_INDEX_KEY) == DF_ON) {
-     insNode->setAvoidHalloween(TRUE);
-     insNode->setHalloweenCannotUseDP2Locks(TRUE);
-   }
-   // used to convey updated columns to insert node's stoi
-   // during inlining the update node is not present anymore, we read the 
-   // insert's stoi to figure out which columns are updated.
-   SqlTableOpenInfo * scanStoi = getLeftmostScanNode()->getOptStoi()->getStoi();
-   short updateColsCount = getOptStoi()->getStoi()->getColumnListCount();
-   scanStoi->setColumnListCount(updateColsCount);
-   scanStoi->setColumnList(new (bindWA->wHeap()) short[updateColsCount]);
-   for (short i=0; i<updateColsCount; i++)
-    scanStoi->setUpdateColumn(i,getOptStoi()->getStoi()->getUpdateColumn(i));
-
-   boundExpr = insNode->bindNode(bindWA);
-   return boundExpr;
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/optimizer/RelUpdate.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelUpdate.h b/core/sql/optimizer/RelUpdate.h
index 38b1729..1e0b9cd 100644
--- a/core/sql/optimizer/RelUpdate.h
+++ b/core/sql/optimizer/RelUpdate.h
@@ -1438,10 +1438,6 @@ private:
   // inlining for the new insert and delete nodes)
   RelExpr *transformUpdatePrimaryKey(BindWA *bindWA);
 
-  // right now, hbase updates are done as cursor update...select followed
-  // by deletes.
-  RelExpr *transformHbaseUpdate(BindWA *bindWA);
-
   inline void setEstRowsAccessed(CostScalar r)  { estRowsAccessed_ = r; }
 
   // Estimated number of rows accessed by Update operator.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/regress/executor/EXPECTED015.SB
----------------------------------------------------------------------
diff --git a/core/sql/regress/executor/EXPECTED015.SB b/core/sql/regress/executor/EXPECTED015.SB
index 9da486e..a7df8af 100755
--- a/core/sql/regress/executor/EXPECTED015.SB
+++ b/core/sql/regress/executor/EXPECTED015.SB
@@ -2236,4 +2236,154 @@ A                     B
 >>cqd traf_upsert_to_eff_tree reset;
 
 --- SQL operation complete.
+>>
+>>-- merge tests (and insert and update tests) on tables with check constraints
+>>insert into t015tc1 values ('ACID','FOUR','GUNK'), ('FIVE','FIVE','HIVE'),('TENT','TEXT','TEST');
+
+--- 3 row(s) inserted.
+>>select * From t015tc1;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  HIVE
+TENT  TEXT  TEST
+
+--- 3 row(s) selected.
+>>
+>>insert into t015tc2 values ('ACID','FOUR','GUNK'), ('FIVE','FIVE','HIVE'),('TENT','TEXT','TEST');
+
+--- 3 row(s) inserted.
+>>select * From t015tc2;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  HIVE
+TENT  TEXT  TEST
+
+--- 3 row(s) selected.
+>>
+>>-- should fail, constraint violation
+>>merge into t015tc1 on a='FIVE' when matched then update set c= 'BALD';
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_454758687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) updated.
+>>
+>>-- should fail, constraint violation
+>>merge into t015tc1 on a = 'TENT' when matched then update set b = 'ACID';
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_318658687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) updated.
+>>
+>>select * From t015tc1;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  HIVE
+TENT  TEXT  TEST
+
+--- 3 row(s) selected.
+>>
+>>-- no constraint violation
+>>merge into t015tc1 on a='FIVE' when matched then update set c= 'GLAD';
+
+--- 1 row(s) updated.
+>>
+>>-- should fail because 'BOMB' < 'FUNK'
+>>merge into t015tc1 on a = 'NOSH' when matched then update set b = 'GOOD'
++>  when not matched then insert values ('GOOD','GOOD','BOMB');
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_454758687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) updated.
+>>
+>>-- the row GOOD, GOOD, NICE should successfully be added
+>>merge into t015tc1 on a = 'NOSH' when matched then update set b = 'GOOD'
++>  when not matched then insert values ('GOOD','GOOD','NICE');
+
+--- 1 row(s) updated.
+>>
+>>select * from t015tc1;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  GLAD
+GOOD  GOOD  NICE
+TENT  TEXT  TEST
+
+--- 4 row(s) selected.
+>>
+>>-- should fail because of constraint on C
+>>insert into t015tc1 values ('ACID','TEST','ACID');
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_454758687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) inserted.
+>>
+>>-- should fail because of constraint on B
+>>insert into t015tc1 values ('ACID','ACID','TEST');
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_318658687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) inserted.
+>>
+>>-- should fail because of constraint on B
+>>update t015tc1 set a = 'COOL', b = 'ACID' where a = 'TENT';
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC1_318658687_3491 on table TRAFODION.SCH.T015TC1.
+
+--- 0 row(s) updated.
+>>
+>>select * from t015tc1;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  GLAD
+GOOD  GOOD  NICE
+TENT  TEXT  TEST
+
+--- 4 row(s) selected.
+>>
+>>-- should fail because of constraint on A
+>>insert into t015tc2 values ('NUMB','NUMB','NUMB');
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC2_289858687_3491 on table TRAFODION.SCH.T015TC2.
+
+--- 0 row(s) inserted.
+>>
+>>-- should fail because of constraint on A
+>>update t015tc2 set a = 'NUMB', b = 'OKAY' where a = 'TENT';
+
+*** ERROR[8101] The operation is prevented by check constraint TRAFODION.SCH.T015TC2_289858687_3491 on table TRAFODION.SCH.T015TC2.
+
+--- 0 row(s) updated.
+>>
+>>-- should fail because of constraint on B
+>>merge into t015tc1 on a = 'NOSH' when matched then update set b = 'ACID';
+
+--- 0 row(s) updated.
+>>
+>>select * from t015tc2;
+
+A     B     C   
+----  ----  ----
+
+ACID  FOUR  GUNK
+FIVE  FIVE  HIVE
+TENT  TEXT  TEST
+
+--- 3 row(s) selected.
+>>
+>>
 >>log;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/c22b6d10/core/sql/regress/executor/TEST015
----------------------------------------------------------------------
diff --git a/core/sql/regress/executor/TEST015 b/core/sql/regress/executor/TEST015
index 734d8f8..57ddbcb 100755
--- a/core/sql/regress/executor/TEST015
+++ b/core/sql/regress/executor/TEST015
@@ -49,6 +49,8 @@ drop table odt cascade;
 drop table de cascade ;
 drop table dec1 cascade;
 drop table t015t16 cascade;
+drop table t015tc1;
+drop table t015tc2;
 
 --control query default POS 'OFF';
 
@@ -164,6 +166,14 @@ create table t015t13 (a largeint generated always as identity, b int not null, c
 create table t015t14 (a int not null primary key, b int not null unique);
 create table t015t15 (a int not null primary key, constraint cons1 foreign key (a) references t015t14(b)) ;
 
+-- a table with check constraints on non-key columns
+create table t015tc1 (a char(4) not null not droppable primary key, b char(4) not null, c char(4) not null, 
+                      check (b > 'BASE'), check (c > 'FUNK'));
+
+-- a table with check constraints on key and non-key columns
+create table t015tc2 (a char(4) not null not droppable primary key, b char(4) not null, c char(4) not null, 
+                      check (a <> 'NUMB'), check (b > 'BASE'), check (c > 'FUNK'));
+
 set session default ESP_RELEASE_WORK_TIMEOUT '-2';
 log log015 clear;
 
@@ -949,5 +959,57 @@ execute xx2;
 select a,b from t015t16 order by a;
 
 cqd traf_upsert_to_eff_tree reset;
+
+-- merge tests (and insert and update tests) on tables with check constraints
+insert into t015tc1 values ('ACID','FOUR','GUNK'), ('FIVE','FIVE','HIVE'),('TENT','TEXT','TEST');
+select * From t015tc1;
+
+insert into t015tc2 values ('ACID','FOUR','GUNK'), ('FIVE','FIVE','HIVE'),('TENT','TEXT','TEST');
+select * From t015tc2;
+
+-- should fail, constraint violation
+merge into t015tc1 on a='FIVE' when matched then update set c= 'BALD';
+
+-- should fail, constraint violation
+merge into t015tc1 on a = 'TENT' when matched then update set b = 'ACID';
+
+select * From t015tc1; 
+
+-- no constraint violation
+merge into t015tc1 on a='FIVE' when matched then update set c= 'GLAD';
+
+-- should fail because 'BOMB' < 'FUNK'
+merge into t015tc1 on a = 'NOSH' when matched then update set b = 'GOOD'
+  when not matched then insert values ('GOOD','GOOD','BOMB');
+
+-- the row GOOD, GOOD, NICE should successfully be added
+merge into t015tc1 on a = 'NOSH' when matched then update set b = 'GOOD'
+  when not matched then insert values ('GOOD','GOOD','NICE');
+
+select * from t015tc1;
+
+-- should fail because of constraint on C
+insert into t015tc1 values ('ACID','TEST','ACID');
+
+-- should fail because of constraint on B
+insert into t015tc1 values ('ACID','ACID','TEST');
+
+-- should fail because of constraint on B
+update t015tc1 set a = 'COOL', b = 'ACID' where a = 'TENT';
+
+select * from t015tc1;
+
+-- should fail because of constraint on A
+insert into t015tc2 values ('NUMB','NUMB','NUMB');
+
+-- should fail because of constraint on A
+update t015tc2 set a = 'NUMB', b = 'OKAY' where a = 'TENT';
+
+-- should fail because of constraint on B
+merge into t015tc1 on a = 'NOSH' when matched then update set b = 'ACID';
+
+select * from t015tc2;
+
+
 log;
 


[3/5] incubator-trafodion git commit: Reworks based on Hans review comments

Posted by db...@apache.org.
Reworks based on Hans 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/1877b7d5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/1877b7d5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/1877b7d5

Branch: refs/heads/master
Commit: 1877b7d5be8337777d16f7603b55dd6bfc6e638a
Parents: 6355b69
Author: Dave Birdsall <db...@apache.org>
Authored: Wed Oct 4 20:55:15 2017 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Wed Oct 4 20:55:15 2017 +0000

----------------------------------------------------------------------
 core/sql/generator/GenRelUpdate.cpp | 101 +++++++++++++++++--------------
 core/sql/optimizer/ValueDesc.h      |   2 +
 core/sql/sqlcomp/DefaultConstants.h |   2 -
 core/sql/sqlcomp/nadefaults.cpp     |   1 -
 4 files changed, 58 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1877b7d5/core/sql/generator/GenRelUpdate.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelUpdate.cpp b/core/sql/generator/GenRelUpdate.cpp
index acfe7f9..9cda158 100644
--- a/core/sql/generator/GenRelUpdate.cpp
+++ b/core/sql/generator/GenRelUpdate.cpp
@@ -301,22 +301,21 @@ static short genUpdConstraintExpr(Generator * generator,
 {
   ExpGenerator * expGen = generator->getExpGenerator();
 
-  // The Attributes for the table columns refer to the old values of the column.
-  // The constraints must operate on the new values, though. So we must do a
-  // switcheroo on the Attributes for the update expression. The target value IDs
-  // come from targetRecExprArray.
-
-  ValueIdList savedSourceVIDlist;
-  NAList<Attributes*> savedSourceAttrsList(generator->wHeap());
+  // The ValueIds in the constrTree refer to the source values of the columns.
+  // Construct a ValueIdMap so we can rewrite the constrTree to refer to the
+  // target value of the columns.
 
+  ValueIdMap sourceToTarget;  // top values will be source, bottom will be target
+ 
   for (ValueId sourceValId = constraintColumns.init();
        constraintColumns.next(sourceValId);
        constraintColumns.advance(sourceValId))
     {
+      GenAssert(sourceValId.getItemExpr()->getOperatorType() == ITM_INDEXCOLUMN,
+      		"unexpected type of constraint expression column");
       NAColumn * sourceCol = ((IndexColumn*)sourceValId.getItemExpr())->getNAColumn();
       ValueId targetValId;
-      NABoolean found = FALSE;
-      for (CollIndex ni = 0; (!found) && (ni < targetRecExprArray.entries()); ni++)
+      for (CollIndex ni = 0; (ni < targetRecExprArray.entries()); ni++)
         {
           const ItemExpr *assignExpr = targetRecExprArray[ni].getItemExpr();
           targetValId = assignExpr->child(0)->castToItemExpr()->getValueId();
@@ -325,50 +324,62 @@ static short genUpdConstraintExpr(Generator * generator,
             targetCol = ((BaseColumn*)targetValId.getItemExpr())->getNAColumn();
           else if (targetValId.getItemExpr()->getOperatorType() == ITM_INDEXCOLUMN)
             targetCol = ((IndexColumn*)targetValId.getItemExpr())->getNAColumn();
-                
-          if ((targetCol) &&
-              (targetCol->getColName() == sourceCol->getColName()) &&
-              (targetCol->getHbaseColFam() == sourceCol->getHbaseColFam()) &&
-              (targetCol->getHbaseColQual() == sourceCol->getHbaseColQual()) &&
-              (targetCol->getNATable()->getTableName().getQualifiedNameAsAnsiString() ==
-               sourceCol->getNATable()->getTableName().getQualifiedNameAsAnsiString()))
+
+          if (targetCol && sourceCol->getPosition() == targetCol->getPosition())
             {
-              found = TRUE;
-              break;
+              GenAssert(sourceCol->getNATable() == targetCol->getNATable(),
+                        "expecting same NATable for constraint source and target");             
+
+              // We found the target column matching the source column in the
+              // targetRecExprArray. Now, an optimization: If the assignment
+              // merely moves the old column value to the new, there is no need
+              // to map it.
+
+              ValueId rhsValId = assignExpr->child(1)->castToItemExpr()->getValueId();
+              NAColumn *rhsCol = NULL;
+              if (rhsValId.getItemExpr()->getOperatorType() == ITM_BASECOLUMN)
+                rhsCol = ((BaseColumn*)rhsValId.getItemExpr())->getNAColumn();
+              else if (rhsValId.getItemExpr()->getOperatorType() == ITM_INDEXCOLUMN)
+                rhsCol = ((IndexColumn*)rhsValId.getItemExpr())->getNAColumn();
+
+              if (rhsCol && rhsCol->getPosition() == targetCol->getPosition())
+                {
+                  // assignment copies old column value to target without change;
+                  // no need to map
+                  GenAssert(rhsCol->getNATable() == targetCol->getNATable(),
+                            "expecting same NATable for assignment source and target");
+                }
+              else
+                {
+                  // the column value is changing (or maybe this is an insert),
+                  // so map it
+                  sourceToTarget.addMapEntry(sourceValId, targetValId);
+                }
+              ni = targetRecExprArray.entries();  // found it, no need to search further
             }
         }
+    } 
 
-      if (found)
-	{
-          Attributes * sourceValAttr = (generator->addMapInfo(sourceValId, 0))->getAttr();
-          Attributes * targetValAttr = (generator->getMapInfo(targetValId, 0))->getAttr();
+  // If there is anything to map, rewrite the constraint expression
+  // and generate it. If there is nothing to map, that means none of
+  // the constraint expression columns is changed (which implies this
+  // is an update expr and not an insert, by the way). In that case, we
+  // don't need to generate the constraint expression as the constraint
+  // should already be satisfied by the old values.
 
-          // Save original location attributes so we can change them back after
-          // generating the update constraint expression
-
-          Attributes * savedValAttr = new(generator->wHeap()) Attributes();
-          savedValAttr->copyLocationAttrs(sourceValAttr);
-          savedSourceAttrsList.insert(savedValAttr);
-          savedSourceVIDlist.insert(sourceValId);
-
-          sourceValAttr->copyLocationAttrs(targetValAttr);
-        }
+  if (sourceToTarget.entries() > 0)
+    {
+      // map the ValueIds in the constraint tree to target values
+      ValueId mappedConstrTree;
+      sourceToTarget.rewriteValueIdDown(constrTree->getValueId(),mappedConstrTree /* out */);
 
+      // generate the expression
+      expGen->generateExpr(mappedConstrTree, ex_expr::exp_SCAN_PRED,
+                           targetExpr);
     }
-
-  // Now that we have remapped the Attributes for the columns to their values
-  // in the new record, we can generate the update constraint expression.
-
-  expGen->generateExpr(constrTree->getValueId(), ex_expr::exp_SCAN_PRED,
-                       targetExpr);
-
-  // Now put the Attributes back the way they were.
-
-  for (Lng32 i = 0; i < savedSourceVIDlist.entries(); i++)
+  else
     {
-      ValueId sourceValId = savedSourceVIDlist[i];
-      Attributes * sourceValAttr = (generator->getMapInfo(sourceValId, 0))->getAttr();
-      sourceValAttr->copyLocationAttrs(savedSourceAttrsList[i]);
+      targetExpr = NULL;
     }
 
   return 0;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1877b7d5/core/sql/optimizer/ValueDesc.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/ValueDesc.h b/core/sql/optimizer/ValueDesc.h
index 11b1be1..1074749 100644
--- a/core/sql/optimizer/ValueDesc.h
+++ b/core/sql/optimizer/ValueDesc.h
@@ -1533,6 +1533,8 @@ public:
   NABoolean operator != (const ValueIdMap &other) const
                                          { return NOT operator==(other); }
 
+  CollIndex entries() { return topValues_.entries(); }
+
   // accessor functions
   const ValueIdList & getTopValues() const    { return topValues_; }
   const ValueIdList & getBottomValues() const { return bottomValues_; }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1877b7d5/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/DefaultConstants.h b/core/sql/sqlcomp/DefaultConstants.h
index 3852e70..e077ce7 100644
--- a/core/sql/sqlcomp/DefaultConstants.h
+++ b/core/sql/sqlcomp/DefaultConstants.h
@@ -3350,8 +3350,6 @@ enum DefaultConstants
   HBASE_MAX_NUM_SEARCH_KEYS,
   CALL_EMBEDDED_ARKCMP,
 
-  HBASE_TRANSFORM_UPDATE_TO_DELETE_INSERT,
-
   // ON, if blocks should be cached for this scan. This includes subset scans used by Update and Delete 
   HBASE_CACHE_BLOCKS,
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1877b7d5/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/nadefaults.cpp b/core/sql/sqlcomp/nadefaults.cpp
index 382d430..1d3cb76 100644
--- a/core/sql/sqlcomp/nadefaults.cpp
+++ b/core/sql/sqlcomp/nadefaults.cpp
@@ -1782,7 +1782,6 @@ SDDkwd__(EXE_DIAGNOSTIC_EVENTS,		"OFF"),
   DDkwd__(HBASE_SMALL_SCANNER,      "OFF"),
   DDkwd__(HBASE_SQL_IUD_SEMANTICS,		"ON"),
   DDkwd__(HBASE_STATS_PARTITIONING,           	"ON"),
-  DDkwd__(HBASE_TRANSFORM_UPDATE_TO_DELETE_INSERT,		"OFF"),
 
   // If set to 'OFF' we get a stub cost of 1 for update operations.
   // We can remove this once the delete costing code has broader


[2/5] incubator-trafodion git commit: Fix bug noticed on pull request review

Posted by db...@apache.org.
Fix bug noticed on pull request review


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

Branch: refs/heads/master
Commit: 6355b6909de9690020794b53ede7410a769dfd77
Parents: c22b6d1
Author: Dave Birdsall <db...@apache.org>
Authored: Tue Oct 3 17:05:22 2017 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Tue Oct 3 17:05:22 2017 +0000

----------------------------------------------------------------------
 core/sql/generator/GenRelUpdate.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/6355b690/core/sql/generator/GenRelUpdate.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelUpdate.cpp b/core/sql/generator/GenRelUpdate.cpp
index 9b9b4e7..acfe7f9 100644
--- a/core/sql/generator/GenRelUpdate.cpp
+++ b/core/sql/generator/GenRelUpdate.cpp
@@ -315,12 +315,12 @@ static short genUpdConstraintExpr(Generator * generator,
     {
       NAColumn * sourceCol = ((IndexColumn*)sourceValId.getItemExpr())->getNAColumn();
       ValueId targetValId;
-      NAColumn *targetCol = NULL;
       NABoolean found = FALSE;
       for (CollIndex ni = 0; (!found) && (ni < targetRecExprArray.entries()); ni++)
         {
           const ItemExpr *assignExpr = targetRecExprArray[ni].getItemExpr();
-          targetValId = assignExpr->child(0)->castToItemExpr()->getValueId();            
+          targetValId = assignExpr->child(0)->castToItemExpr()->getValueId();
+          NAColumn *targetCol = NULL;         
           if (targetValId.getItemExpr()->getOperatorType() == ITM_BASECOLUMN)
             targetCol = ((BaseColumn*)targetValId.getItemExpr())->getNAColumn();
           else if (targetValId.getItemExpr()->getOperatorType() == ITM_INDEXCOLUMN)


[4/5] incubator-trafodion git commit: Rework some comments

Posted by db...@apache.org.
Rework some 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/4e9d5f3f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/4e9d5f3f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/4e9d5f3f

Branch: refs/heads/master
Commit: 4e9d5f3f0b47e9dfc87551644cf16729d7f849a2
Parents: 1877b7d
Author: Dave Birdsall <db...@apache.org>
Authored: Wed Oct 4 22:30:01 2017 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Wed Oct 4 22:30:01 2017 +0000

----------------------------------------------------------------------
 core/sql/generator/GenRelUpdate.cpp | 18 +++++++-----------
 1 file changed, 7 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e9d5f3f/core/sql/generator/GenRelUpdate.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenRelUpdate.cpp b/core/sql/generator/GenRelUpdate.cpp
index 9cda158..2c93d69 100644
--- a/core/sql/generator/GenRelUpdate.cpp
+++ b/core/sql/generator/GenRelUpdate.cpp
@@ -1808,13 +1808,11 @@ short HbaseUpdate::codeGen(Generator * generator)
     {
       // Generate the update and insert constraint check expressions
 
-      // The attributes for the columns referenced in the constraint expressions
-      // refer to the source values of the columns. We want to evaluate the
-      // constraints aganst the target values, though. So, there is some
-      // Attributes gymnastics that has to happen to generate them.
-
-      // Obtain the ValueIds of base table columns referenced in the
-      // constraints
+      // The constraint expressions at this time refer to the source values 
+      // of the columns. We want to evaluate the constraints aganst the target 
+      // values, though. So, we collect the source column ValueIds here so
+      // we can map them to the appropriate target, which is dependent on
+      // which constraint expression we are generating.
 
       ValueId constraintId;
       ValueIdSet constraintColumns;
@@ -1835,8 +1833,7 @@ short HbaseUpdate::codeGen(Generator * generator)
       if (getTableDesc()->getNATable()->hasSerializedEncodedColumn())
         constrTree = generator->addCompDecodeForDerialization(constrTree, isAlignedFormat);
 
-      // Generate the update constraint expression, substituting Attributes for
-      // the new update record
+      // Generate the update constraint expression
 
       genUpdConstraintExpr(generator,
                            constrTree,
@@ -1846,8 +1843,7 @@ short HbaseUpdate::codeGen(Generator * generator)
 
       if ((isMerge()) && (mergeInsertRecExprArray().entries() > 0))
         {
-          // Generate the insert constraint expression, substituting Attributes for
-          // the new insert record
+          // Generate the insert constraint expression
 
           genUpdConstraintExpr(generator,
                                constrTree,


[5/5] incubator-trafodion git commit: Merge [TRAFODION-1610] PR 1253 Generate constraint exprs on update operators

Posted by db...@apache.org.
Merge [TRAFODION-1610] PR 1253 Generate constraint exprs on update operators


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

Branch: refs/heads/master
Commit: f9ba966ed8f6f246c468df46204a86226efcade6
Parents: 4706888 4e9d5f3
Author: Dave Birdsall <db...@apache.org>
Authored: Thu Oct 5 15:22:56 2017 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Thu Oct 5 15:22:56 2017 +0000

----------------------------------------------------------------------
 core/sql/generator/GenRelUpdate.cpp      | 210 ++++++++++++++++----------
 core/sql/optimizer/BindRelExpr.cpp       |  28 ----
 core/sql/optimizer/Inlining.cpp          | 104 -------------
 core/sql/optimizer/RelUpdate.h           |   4 -
 core/sql/optimizer/ValueDesc.h           |   2 +
 core/sql/regress/executor/EXPECTED015.SB | 150 ++++++++++++++++++
 core/sql/regress/executor/TEST015        |  62 ++++++++
 core/sql/sqlcomp/DefaultConstants.h      |   2 -
 core/sql/sqlcomp/nadefaults.cpp          |   1 -
 9 files changed, 344 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f9ba966e/core/sql/optimizer/Inlining.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f9ba966e/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f9ba966e/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------