You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by hz...@apache.org on 2017/10/04 01:24:31 UTC

[5/7] incubator-trafodion git commit: [TRAFODION-2761] Error when all TMUDF columns are eliminated

[TRAFODION-2761] Error when all TMUDF columns are eliminated

Fix a couple of places where we would core dump when all
columns of the input table of a TMUDF were eliminated. Also
removed an unused NAColumnArray method that has a bug.


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

Branch: refs/heads/master
Commit: 4e7ad3aa967a60d4a212074e962f0f22a811eaf0
Parents: 4b1c64c
Author: Hans Zeller <hz...@apache.org>
Authored: Tue Oct 3 17:04:12 2017 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Tue Oct 3 17:04:12 2017 +0000

----------------------------------------------------------------------
 core/sql/executor/ExUdr.cpp                  |   5 +-
 core/sql/generator/GenUdr.cpp                |   3 +-
 core/sql/optimizer/NAColumn.cpp              |  14 ---
 core/sql/optimizer/NAColumn.h                |   1 -
 core/sql/regress/udr/EXPECTED001             |  24 +++-
 core/sql/regress/udr/TEST001                 |  13 ++
 core/sql/regress/udr/TEST001_Sessionize.java | 142 +++++++++++++++-------
 7 files changed, 136 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/executor/ExUdr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/ExUdr.cpp b/core/sql/executor/ExUdr.cpp
index 24b6aaf..230aa42 100644
--- a/core/sql/executor/ExUdr.cpp
+++ b/core/sql/executor/ExUdr.cpp
@@ -452,8 +452,9 @@ ExUdrTcb::ExUdrTcb(const ExUdrTdb &udrTdb,
   Int32 i;
   for (i=0; i<numChildren(); i++)
     {
-      udrTdb.getChildInputExpr(i)->fixup(0,getExpressionMode(),this,
-				  globSpace, globHeap, FALSE, glob);
+      if (udrTdb.getChildInputExpr(i))
+        udrTdb.getChildInputExpr(i)->fixup(0,getExpressionMode(),this,
+                                           globSpace, globHeap, FALSE, glob);
     }
   if (udrTdb.getPredicate())
     udrTdb.getPredicate()->fixup(0, getExpressionMode(), this,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/generator/GenUdr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenUdr.cpp b/core/sql/generator/GenUdr.cpp
index 682ab6a..96767b3 100644
--- a/core/sql/generator/GenUdr.cpp
+++ b/core/sql/generator/GenUdr.cpp
@@ -555,8 +555,7 @@ static short udr_codegen(Generator *generator,
 
       childInfo = op->getChildInfo(0);
       numChildInputCols = childInfo->getOutputs().entries();
-      if (numChildInputCols > 0)
-        childInput_exprs = new(space)ex_expr*[1];
+      childInput_exprs = new(space)ex_expr*[1];
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/optimizer/NAColumn.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NAColumn.cpp b/core/sql/optimizer/NAColumn.cpp
index 38d8dde..322a3a4 100644
--- a/core/sql/optimizer/NAColumn.cpp
+++ b/core/sql/optimizer/NAColumn.cpp
@@ -741,19 +741,6 @@ NAColumn * NAColumnArray::getColumnByPos(Lng32 position) const
   return NULL;
 }
 
-// removes the column that has the same position
-void NAColumnArray::removeByPosition(Lng32 position)
-{
-  for(CollIndex i=0;i < entries();i++)
-  {
-    NAColumn * column = (*this)[i];
-    if(column->getPosition() == position)
-    {
-      this->removeAt(i);
-      break;
-    }
-  }
-}
 Lng32 NAColumnArray::getOffset(Lng32 position) const
 {
   Lng32 result = 0;
@@ -789,7 +776,6 @@ ULng32 NAColumnArray::getMaxTrafHbaseColQualifier() const
   return maxVal;
 }
 
-
 //method to reset an NAColumn object after a statement
 //In the ideal world NAColumn objects should not be having any state
 //that changes over the course of a statement, since it represents metadata

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/optimizer/NAColumn.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NAColumn.h b/core/sql/optimizer/NAColumn.h
index b1250d8..a1ea885 100644
--- a/core/sql/optimizer/NAColumn.h
+++ b/core/sql/optimizer/NAColumn.h
@@ -640,7 +640,6 @@ public:
   NAColumn * getColumn(Lng32 index) const;	// same as nacolarray[position]
   NAColumn * getColumn(const char* colName) const;
   NAColumn * getColumnByPos(Lng32 position) const;
-  void removeByPosition(Lng32 position);
 
   // return 
   //    i (i>=0) if the column is found in the array via NAColumn::operator==

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/regress/udr/EXPECTED001
----------------------------------------------------------------------
diff --git a/core/sql/regress/udr/EXPECTED001 b/core/sql/regress/udr/EXPECTED001
index 0c97e87..66c21f7 100644
--- a/core/sql/regress/udr/EXPECTED001
+++ b/core/sql/regress/udr/EXPECTED001
@@ -519,7 +519,29 @@ IPADDR           SESSION_ID            SEQUENCE_NO
 SESSION_ID            SEQUENCE_NO           USERID                            TS                    IPADDR
 --------------------  --------------------  --------------------------------  --------------------  ---------------
 
-                   1                     1  super-user                          212334861599500000  12.345.567.345 
+                   1                     1  super-user                          212373741599500000  12.345.567.345 
+
+--- 1 row(s) selected.
+>>
+>>-- eliminate all columns
+>>prepare s from
++>SELECT COUNT(*)
++>FROM UDF(sessionize_java(TABLE(SELECT userid,
++>                                      JULIANTIMESTAMP(ts) as TS,
++>                                      ipAddr
++>                               FROM clicks
++>                               PARTITION BY ipaddr ORDER BY ts),
++>                         'IPADDR',
++>                         cast('TS' as char(2)),
++>                         60000000)) XO;
+
+--- SQL command prepared.
+>>execute s;
+
+(EXPR)              
+--------------------
+
+                   4
 
 --- 1 row(s) selected.
 >>

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/regress/udr/TEST001
----------------------------------------------------------------------
diff --git a/core/sql/regress/udr/TEST001 b/core/sql/regress/udr/TEST001
index a4dc660..395cee8 100644
--- a/core/sql/regress/udr/TEST001
+++ b/core/sql/regress/udr/TEST001
@@ -366,6 +366,19 @@ where operator = 'TMUDF'
   and description like '% preds_evaluated_by_udf: (SESSION_ID < 2) %';
 execute s;
 
+-- eliminate all columns
+prepare s from
+SELECT COUNT(*)
+FROM UDF(sessionize_java(TABLE(SELECT userid,
+                                      JULIANTIMESTAMP(ts) as TS,
+                                      ipAddr
+                               FROM clicks
+                               PARTITION BY ipaddr ORDER BY ts),
+                         'IPADDR',
+                         cast('TS' as char(2)),
+                         60000000)) XO;
+execute s;
+
 -- will fail until tinyint support is added for spj/procedures
 cqd traf_tinyint_spj_support 'ON';
 select *

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/4e7ad3aa/core/sql/regress/udr/TEST001_Sessionize.java
----------------------------------------------------------------------
diff --git a/core/sql/regress/udr/TEST001_Sessionize.java b/core/sql/regress/udr/TEST001_Sessionize.java
index 74217b6..16267d2 100644
--- a/core/sql/regress/udr/TEST001_Sessionize.java
+++ b/core/sql/regress/udr/TEST001_Sessionize.java
@@ -40,6 +40,29 @@ class TEST001_Sessionize extends UDR
     public TEST001_Sessionize()
     {}
 
+    boolean generatedColumnsAreUsed(UDRInvocationInfo info)
+    {
+        boolean result = false;
+
+        try {
+            if (info.out().getColumn("SESSION_ID").getUsage() ==
+                ColumnInfo.ColumnUseCode.USED)
+                result = true;
+        }
+        catch (UDRException e) {
+        }
+
+        try {
+            if (info.out().getColumn("SEQUENCE_NO").getUsage() ==
+                ColumnInfo.ColumnUseCode.USED)
+                result = true;
+        }
+        catch (UDRException e) {
+        }
+
+        return result;
+    }
+
     // determine output columns dynamically at compile time
     @Override
     public void describeParamsAndColumns(UDRInvocationInfo info)
@@ -160,24 +183,31 @@ class TEST001_Sessionize extends UDR
         // column as unused, however. So, make sure these two
         // columns are definitely included.
 
-        // first, recompute the id and timestamp column numbers
-        InternalColumns state = new TEST001_Sessionize.InternalColumns(
-            info.in().getColNum(info.par().getString(0)),
-            info.in().getColNum(info.par().getString(1)));
+        boolean genColsAreUsed = generatedColumnsAreUsed(info);
 
-        // then include the columns
-        info.setChildColumnUsage(0, state.getIdColumn(), ColumnInfo.ColumnUseCode.USED);
-        info.setChildColumnUsage(0, state.getTsColumn(), ColumnInfo.ColumnUseCode.USED);
-
-        boolean generatedColsAreUsed =
-            (info.out().getColumn(0).getUsage() == ColumnInfo.ColumnUseCode.USED ||
-             info.out().getColumn(1).getUsage() == ColumnInfo.ColumnUseCode.USED);
+        if (genColsAreUsed)
+            {
+                // first, recompute the id and timestamp column numbers
+                InternalColumns state = new TEST001_Sessionize.InternalColumns(
+                    info.in().getColNum(info.par().getString(0)),
+                    info.in().getColNum(info.par().getString(1)));
+
+                // second, include the id/timestamp columns
+                info.setChildColumnUsage(0, state.getIdColumn(), ColumnInfo.ColumnUseCode.USED);
+                info.setChildColumnUsage(0, state.getTsColumn(), ColumnInfo.ColumnUseCode.USED);
+
+                // third, if any of the generated columns is needed, then produce them all,
+                // this "all or none" approach just makes our life a bit easier
+                for (int i=0; i<2; i++)
+                  if (info.out().getColumn(i).getUsage() == ColumnInfo.ColumnUseCode.NOT_PRODUCED)
+                    info.out().getColumn(i).setUsage(ColumnInfo.ColumnUseCode.NOT_USED);
+            }
 
         // Walk through predicates and find additional ones to push down
         // or to evaluate locally
         for (int p=0; p<info.getNumPredicates(); p++)
           {
-            if (!generatedColsAreUsed)
+            if (!genColsAreUsed)
               {
                   // If session_id/sequence_no are not used in the query, then
                   // we can push all predicates to the children.
@@ -207,15 +237,19 @@ class TEST001_Sessionize extends UDR
         // its input tables to the result.
         info.propagateConstraintsFor1To1UDFs(false);
 
-        // The id column, together with session id and sequence_no, form a unique key.
-        // Generate a uniqueness constraint for that.
+        if (generatedColumnsAreUsed(info))
+          {
+            // The id column, together with session id and
+            // sequence_no, form a unique key.  Generate a
+            // uniqueness constraint for that.
 
-        UniqueConstraintInfo uc = new UniqueConstraintInfo();
+            UniqueConstraintInfo uc = new UniqueConstraintInfo();
 
-        uc.addColumn(info.out().getColNum(info.par().getString(0)));
-        uc.addColumn(0); // the session id is alway column #0
-        uc.addColumn(1); // the sequence number alway column #1
-        info.out().addUniquenessConstraint(uc);
+            uc.addColumn(info.out().getColNum(info.par().getString(0)));
+            uc.addColumn(0); // the session id is always column #0
+            uc.addColumn(1); // the sequence number is always column #1
+            info.out().addUniquenessConstraint(uc);
+          }
     }
 
     // estimate result cardinality
@@ -258,10 +292,20 @@ class TEST001_Sessionize extends UDR
                             UDRPlanInfo plan)
         throws UDRException
     {
+        boolean genColsAreUsed = generatedColumnsAreUsed(info);
+
         // read the three parameters and convert the first two into column numbers
-        int userIdColNum    = info.in(0).getColNum(info.par().getString(0));
-        int timeStampColNum = info.in(0).getColNum(info.par().getString(1));
-        long timeout        = info.par().getLong(2);
+        int userIdColNum    = -1;
+        int timeStampColNum = -1;
+        long timeout        = -1;
+
+        if (genColsAreUsed)
+          {
+            // read the three parameters and convert the first two into column numbers
+            userIdColNum    = info.in(0).getColNum(info.par().getString(0));
+            timeStampColNum = info.in(0).getColNum(info.par().getString(1));
+            timeout         = info.par().getLong(2);
+          }
 
         // variables needed for computing the session id
         long lastTimeStamp = 0;
@@ -281,39 +325,45 @@ class TEST001_Sessionize extends UDR
         // loop over input rows
         while (getNextRow(info))
           {
-            long timeStamp = info.in(0).getLong(timeStampColNum);
-            String userId = info.in(0).getString(userIdColNum);
-
-            if (lastUserId.compareTo(userId) != 0)
+            if (genColsAreUsed)
               {
-                // reset timestamp check and start over with session id 0
-                lastTimeStamp = 0;
-                currSessionId = 1;
-                currSequenceNo = 1;
-                lastUserId = userId;
-              }
-
-            long tsDiff = timeStamp - lastTimeStamp;
-
-            if (tsDiff > timeout && lastTimeStamp > 0)
-              {
-                currSessionId++;
-                currSequenceNo = 1;
-              }
-            else if (tsDiff < 0)
-                throw new UDRException(
+                long timeStamp = info.in(0).getLong(timeStampColNum);
+                String userId = info.in(0).getString(userIdColNum);
+
+                if (lastUserId.compareTo(userId) != 0)
+                  {
+                      // reset timestamp check and start over with session id 0
+                      lastTimeStamp = 0;
+                      currSessionId = 1;
+                      currSequenceNo = 1;
+                      lastUserId = userId;
+                  }
+
+                long tsDiff = timeStamp - lastTimeStamp;
+
+                if (tsDiff > timeout && lastTimeStamp > 0)
+                  {
+                    currSessionId++;
+                    currSequenceNo = 1;
+                  }
+                else if (tsDiff < 0)
+                  throw new UDRException(
                     38001,
                     "Got negative or descending timestamps %ld, %ld",
                     lastTimeStamp, timeStamp);
 
-            lastTimeStamp = timeStamp;
+                lastTimeStamp = timeStamp;
+              }
 
             // this evaluates the SQL predicate on SESSION_ID
             if (currSessionId < maxSessionId)
               {
-                // produce session_id and sequence_no output columns
-                info.out().setLong(0, currSessionId);
-                info.out().setLong(1, currSequenceNo);
+                if (genColsAreUsed)
+                  {
+                    // produce session_id and sequence_no output columns
+                    info.out().setLong(0, currSessionId);
+                    info.out().setLong(1, currSequenceNo);
+                  }
 
                 // produce the remaining columns and emit the row
                 info.copyPassThruData();