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 2018/01/11 17:45:43 UTC

[1/6] trafodion git commit: [TRAFODION-2897] Invalid name in MySQL log file when using install_local_hadoop

Repository: trafodion
Updated Branches:
  refs/heads/master fbd91855b -> 72da2cd7d


[TRAFODION-2897] Invalid name in MySQL log file when using install_local_hadoop

Bug reported on the dev list. MySQL does not understand shell syntax.


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

Branch: refs/heads/master
Commit: bb4edd48c909e889297ec0b0940d9d60745fd86c
Parents: 51935dc
Author: Hans Zeller <hz...@apache.org>
Authored: Tue Jan 9 22:07:09 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Tue Jan 9 22:07:09 2018 +0000

----------------------------------------------------------------------
 core/sqf/sql/scripts/install_local_hadoop | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/bb4edd48/core/sqf/sql/scripts/install_local_hadoop
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/install_local_hadoop b/core/sqf/sql/scripts/install_local_hadoop
index fffeb20..7c2b062 100755
--- a/core/sqf/sql/scripts/install_local_hadoop
+++ b/core/sqf/sql/scripts/install_local_hadoop
@@ -1318,7 +1318,7 @@ lc-messages-dir=${MY_SW_ROOT}/mysql/share
 
 # Enable logging by default to help find problems
 general-log=1
-general-log-file=${MY_SW_ROOT}/log/mysql-general.\${HOSTNAME}.log
+general-log-file=${MY_SW_ROOT}/log/mysql-general.${HOSTNAME}.log
 
 EOF
 


[4/6] trafodion git commit: OSIM changes and comment fix for JDBCUDR

Posted by hz...@apache.org.
OSIM changes and comment fix for JDBCUDR

Two small changes w/o JIRA.

Fixes in the OSIM tool related to external Hive tables. Issue an
unregister command before creating a new Hive table in OSIM simulation
mode.  Also get the REGISTER commands from the OSIM files and send
them to Trafodion, not to Hive, as before. Finally, make some minor
changes to showddl (avoid extra commas, fix comments that showed
REGISTER as a Hive command).

Cosmetic changes in the JDBC UDR (fix sandbox location in comment and
use a utility method to get to the sandbox location instead of
hard-coding it.).


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

Branch: refs/heads/master
Commit: 40b1773401ac7bb323114f3cf180ccbfec1e8caa
Parents: 4d521af
Author: Hans Zeller <hz...@apache.org>
Authored: Tue Jan 9 22:11:37 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Tue Jan 9 22:11:37 2018 +0000

----------------------------------------------------------------------
 core/sql/optimizer/OptimizerSimulator.cpp       | 36 ++++++++++++++++----
 core/sql/sqlcomp/CmpDescribe.cpp                | 10 +++---
 .../org/trafodion/sql/udr/predef/JDBCUDR.java   | 10 +++---
 3 files changed, 40 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/40b17734/core/sql/optimizer/OptimizerSimulator.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/OptimizerSimulator.cpp b/core/sql/optimizer/OptimizerSimulator.cpp
index 7b7fa38..53af84d 100644
--- a/core/sql/optimizer/OptimizerSimulator.cpp
+++ b/core/sql/optimizer/OptimizerSimulator.cpp
@@ -947,6 +947,18 @@ void OptimizerSimulator::dropObjects()
               CmpCommon::diags()->mergeAfter(*(cliInterface_->getDiagsArea()));
               raiseOsimException("drop external table: %d", retcode);
           }
+          //unregister hive table
+          NAString unregisterStmt = "UNREGISTER HIVE TABLE IF EXISTS ";
+          unregisterStmt += name;
+          debugMessage("%s\n", unregisterStmt.data());
+          retcode = executeFromMetaContext(unregisterStmt.data());
+          if(retcode < 0)
+          {
+              //suppress errors for now, even with IF EXISTS this will
+              //give an error if the Hive table does not exist
+              //CmpCommon::diags()->mergeAfter(*(cliInterface_->getDiagsArea()));
+              //raiseOsimException("unregister hive table: %d", retcode);
+          }
           //drop hive table
           NAString hiveSchemaName;
           qualName->getHiveSchemaName(hiveSchemaName);
@@ -1048,10 +1060,12 @@ static const char* extractAsComment(const char* header, const NAString & stmt)
     {
         int end = stmt.index('\n', begin);
         if(end > begin)
-        {
-            stmt.extract(begin, end-1, tmp);
-            return tmp.data();
-        }
+          end -= 1;
+        else
+          end = stmt.length()-1;
+
+        stmt.extract(begin, end, tmp);
+        return tmp.data();
     }
     return NULL;
 }
@@ -1156,7 +1170,14 @@ void OptimizerSimulator::loadHiveDDLs()
     while(readHiveStmt(hiveCreateExternalTableSql, statement, comment))
    {
         if(statement.length() > 0) {
-            debugMessage("%s\n", extractAsComment("CREATE EXTERNAL TABLE", statement));
+            // this could be a create external table or just a register table
+            // if this Hive table just has stats but no external table
+            const char *stmtText = extractAsComment("CREATE EXTERNAL TABLE", statement);
+
+            if (!stmtText)
+              stmtText = extractAsComment("REGISTER  HIVE TABLE", statement);
+            debugMessage("%s\n", stmtText);
+
             retcode = executeFromMetaContext(statement.data()); //create hive external table
             if(retcode < 0)
             {
@@ -2943,8 +2964,9 @@ void OptimizerSimulator::dumpHiveTableDDLs()
             for (int i = 0; i < outQueue->numEntries(); i++) {
                 OutputInfo * vi = (OutputInfo*)outQueue->getNext();
                 char * ptr = vi->get(0);
-                //write "CREATE EXTERNAL TABLE" DDL to another file.
-                if(strstr(ptr, "CREATE EXTERNAL TABLE"))
+                //write "CREATE EXTERNAL TABLE" and "REGISTER" DDL to another file.
+                if(strstr(ptr, "CREATE EXTERNAL TABLE") ||
+                   strstr(ptr, "REGISTER /*INTERNAL*/ HIVE TABLE"))
                     inExtDDL = TRUE;
                 if(inExtDDL){
                     (*writeLogStreams_[HIVE_CREATE_EXTERNAL_TABLE]) << ptr << endl;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/40b17734/core/sql/sqlcomp/CmpDescribe.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpDescribe.cpp b/core/sql/sqlcomp/CmpDescribe.cpp
index 28590f4..26b9b3c 100644
--- a/core/sql/sqlcomp/CmpDescribe.cpp
+++ b/core/sql/sqlcomp/CmpDescribe.cpp
@@ -2411,7 +2411,12 @@ short CmpDescribeHiveTable (
     }
 
   if (type == 2)
-    outputShortLine(space, ";");
+    {
+      outputShortLine(space, ";");
+
+      outputShortLine(space," ");
+      outputShortLine(space,"/* Trafodion DDL */");
+    }
 
   // if this hive table is registered in traf metadata, show that.
   if ((type == 2) &&
@@ -2455,9 +2460,6 @@ short CmpDescribeHiveTable (
       QualifiedName qn(extName, 3);
       CorrName cn(qn);
 
-      outputShortLine(space," ");
-      outputShortLine(space,"/* Trafodion DDL */");
- 
       short rc = CmpDescribeSeabaseTable(cn, 
                                          type,
                                          dummyBuf, dummyLen, heap, 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/40b17734/core/sql/src/main/java/org/trafodion/sql/udr/predef/JDBCUDR.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/udr/predef/JDBCUDR.java b/core/sql/src/main/java/org/trafodion/sql/udr/predef/JDBCUDR.java
index 3cdd747..0c6936b 100644
--- a/core/sql/src/main/java/org/trafodion/sql/udr/predef/JDBCUDR.java
+++ b/core/sql/src/main/java/org/trafodion/sql/udr/predef/JDBCUDR.java
@@ -26,8 +26,8 @@ under the License.
  * Invocation (all arguments are strings):
  *
  * select ... from udf(JDBC(
- *    <name of JDBC driver jar>, // Not really needed if the jar is stored in
- *                               // $TRAF_HOME/udr/public/external_libs
+ *    <name of JDBC driver jar>, // file name of the JDBC driver jar, stored
+ *                               // in $TRAF_HOME/udr/public/external_libs
  *    <name of JDBC driver class in the jar>,
  *    <connection string>,
  *    <user name>,
@@ -98,7 +98,7 @@ class JDBCUDR extends UDR
             Path driverJarPath = Paths.get(driverJar_);
 
             // for security reasons, we sandbox the allowed driver jars
-            // into $TRAF_HOME/export/lib/udr/external_libs
+            // into $TRAF_HOME/udr/public/external_libs
             driverJarPath = driverJarPath.normalize();
             if (driverJarPath.isAbsolute())
               {
@@ -107,7 +107,7 @@ class JDBCUDR extends UDR
                   throw new UDRException(
                     38010,
                     "The jar name of the JDBC driver must be a name relative to %s, got %s",
-                    System.getenv("TRAF_HOME")+"/udr/external_libs",
+                    LmUtility.getSandboxRootForUser(null).toString(),
                     driverJar_);
               }
             else
@@ -141,7 +141,7 @@ class JDBCUDR extends UDR
                 38020,
                 "JDBC driver class %s not found. Please make sure the JDBC driver jar is stored in %s. Message: %s",
                 driverClassName_,
-                System.getenv("TRAF_HOME") + "/udr/public/external_libs",
+                LmUtility.getSandboxRootForUser(null).toString(),
                 cnf.getMessage());
           }
           catch (SQLException se) {


[2/6] trafodion git commit: [TRAFODION-2879] Core dump due to reference to deallocated memory for EstLogProp object

Posted by hz...@apache.org.
[TRAFODION-2879] Core dump due to reference to deallocated memory for EstLogProp object

In the optimizer, we often use a define to refer to an "empty input
logical properties" object. The problem was that this object was
allocated on the statement heap but it was owned by the CmpContext
object, which lives longer than the statement heap. During debugging,
we have seen cases where this led to references of deleted memory.

The fix is to move the empty logical properties to the CmpStatement
object.


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

Branch: refs/heads/master
Commit: 90f0b665d54f6b04486e290afd118335167b1d18
Parents: bb4edd4
Author: Hans Zeller <hz...@apache.org>
Authored: Tue Jan 9 22:09:29 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Tue Jan 9 22:09:29 2018 +0000

----------------------------------------------------------------------
 core/sql/arkcmp/CmpContext.cpp   |  3 ---
 core/sql/arkcmp/CmpContext.h     |  8 --------
 core/sql/arkcmp/CmpStatement.cpp | 23 ++++++++++++-----------
 core/sql/arkcmp/CmpStatement.h   |  6 ++++++
 core/sql/common/CmpCommon.h      |  2 +-
 core/sql/optimizer/RelExpr.h     |  1 +
 6 files changed, 20 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/arkcmp/CmpContext.cpp
----------------------------------------------------------------------
diff --git a/core/sql/arkcmp/CmpContext.cpp b/core/sql/arkcmp/CmpContext.cpp
index e5c179e..3108b47 100644
--- a/core/sql/arkcmp/CmpContext.cpp
+++ b/core/sql/arkcmp/CmpContext.cpp
@@ -304,9 +304,6 @@ CmpContext::CmpContext(UInt32 f, CollHeap * h)
   // create dynamic metadata descriptors
   CmpSeabaseDDL cmpSeabaseDDL(heap_);
   cmpSeabaseDDL.createMDdescs(trafMDDescsInfo_);
-
-  emptyInLogProp_ = NULL;
-
 }
 
 // MV

http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/arkcmp/CmpContext.h
----------------------------------------------------------------------
diff --git a/core/sql/arkcmp/CmpContext.h b/core/sql/arkcmp/CmpContext.h
index b875927..5df2eca 100644
--- a/core/sql/arkcmp/CmpContext.h
+++ b/core/sql/arkcmp/CmpContext.h
@@ -449,11 +449,6 @@ public :
   // optimizer cached defaults
   OptDefaults* getOptDefaults() { return optDefaults_; }
 
-  // context global empty input logical property
-  EstLogPropSharedPtr* getGEILP() { return &emptyInLogProp_; }
-  void setGEILP(EstLogPropSharedPtr inLogProp)
-                             { emptyInLogProp_ = inLogProp; }
-    
   MDDescsInfo *getTrafMDDescsInfo() { return trafMDDescsInfo_; }
 
   void setCIClass(CmpContextInfo::CmpContextClassType x) { ciClass_ = x; }
@@ -615,9 +610,6 @@ private:
   // query defaults using during a statement compilation
   OptDefaults* optDefaults_;
 
-  // context global empty input logical property
-  EstLogPropSharedPtr emptyInLogProp_;
-
   MDDescsInfo * trafMDDescsInfo_;
 
   CmpContextInfo::CmpContextClassType ciClass_;

http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/arkcmp/CmpStatement.cpp
----------------------------------------------------------------------
diff --git a/core/sql/arkcmp/CmpStatement.cpp b/core/sql/arkcmp/CmpStatement.cpp
index e16ff5c..4bde511 100644
--- a/core/sql/arkcmp/CmpStatement.cpp
+++ b/core/sql/arkcmp/CmpStatement.cpp
@@ -1515,17 +1515,18 @@ QueryAnalysis* CmpStatement::initQueryAnalysis()
   // do any necessary initialization work here (unless this
   // initialization work fits in the constructor)
 
-  // Initialize the global "empty input logprop".
-  context_->setGEILP(EstLogPropSharedPtr(new (STMTHEAP)
-                              EstLogProp(1,
-                                         NULL,
-                                         EstLogProp::NOT_SEMI_TSJ,
-                                         new (STMTHEAP) CANodeIdSet(),
-                                         TRUE)));
-
-    //++MV
-    // This input cardinality is not estimated , so we keep this knowledge
-    // in a special attribute.
+  // Initialize the global "empty input logprop"
+  if (emptyInLogProp_ == NULL)
+    emptyInLogProp_ = EstLogPropSharedPtr(
+         new (STMTHEAP) EstLogProp(1,
+                                   NULL,
+                                   EstLogProp::NOT_SEMI_TSJ,
+                                   new (STMTHEAP) CANodeIdSet(STMTHEAP),
+                                   TRUE));
+  
+  //++MV
+  // This input cardinality is not estimated , so we keep this knowledge
+  // in a special attribute.
   (*GLOBAL_EMPTY_INPUT_LOGPROP)->setCardinalityEqOne();
 
 #ifdef _DEBUG

http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/arkcmp/CmpStatement.h
----------------------------------------------------------------------
diff --git a/core/sql/arkcmp/CmpStatement.h b/core/sql/arkcmp/CmpStatement.h
index 14b1781..dc0767a 100644
--- a/core/sql/arkcmp/CmpStatement.h
+++ b/core/sql/arkcmp/CmpStatement.h
@@ -230,6 +230,9 @@ public:
   const LIST(CSEInfo *) *getCSEInfoList() const { return cses_; }
   void addCSEInfo(CSEInfo *info);
 
+  // context global empty input logical property
+  EstLogPropSharedPtr* getGEILP() { return &emptyInLogProp_; }
+    
 protected:
   // CmpStatement(const CmpStatement&); please remove this line
   CmpStatement& operator=(const CmpStatement&);
@@ -327,6 +330,9 @@ private:
   // CmpMain::sqlcomp(QueryText, ...
   Int32 numOfCompilationRetries_;
 
+  // context global empty input logical property
+  EstLogPropSharedPtr emptyInLogProp_;
+
 }; // end of CmpStatement
 
 class CmpStatementISP: public CmpStatement

http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/common/CmpCommon.h
----------------------------------------------------------------------
diff --git a/core/sql/common/CmpCommon.h b/core/sql/common/CmpCommon.h
index 74393ad..1caaafa 100644
--- a/core/sql/common/CmpCommon.h
+++ b/core/sql/common/CmpCommon.h
@@ -141,7 +141,7 @@ public:
   #define CURRCONTEXT_OPTDEBUG (CmpCommon::context()->getOptDbg())
   #define CURRCONTEXT_HISTCACHE (CmpCommon::context()->getHistogramCache())
   #define CURRCONTEXT_OPTSIMULATOR (CmpCommon::context()->getOptimizerSimulator())
-  #define GLOBAL_EMPTY_INPUT_LOGPROP (CmpCommon::context()->getGEILP())
+  #define GLOBAL_EMPTY_INPUT_LOGPROP (CmpCommon::statement()->getGEILP())
   #define CURRSTMT_OPTDEFAULTS (CmpCommon::context()->getOptDefaults())
 
   // For some routines that do care about the current CmpContext*. 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/90f0b665/core/sql/optimizer/RelExpr.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExpr.h b/core/sql/optimizer/RelExpr.h
index 3074aa8..44ed086 100644
--- a/core/sql/optimizer/RelExpr.h
+++ b/core/sql/optimizer/RelExpr.h
@@ -39,6 +39,7 @@
 
 #include "ObjectNames.h"
 #include "CmpContext.h"
+#include "CmpStatement.h"
 #include "RETDesc.h"
 #include "ValueDesc.h"
 #include "Rule.h"


[3/6] trafodion git commit: [TRAFODION-2896] Internal error in stored procedures when a warning is generated in SQL

Posted by hz...@apache.org.
[TRAFODION-2896] Internal error in stored procedures when a warning is generated in SQL

This problem was seen by Any on a large cluster and Selva found this issue
during debugging. At this point we don't have an easy testcase to reproduce,
but we agree that the code change is something we should do.


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

Branch: refs/heads/master
Commit: 4d521af7799983c44f93e9ed46ed48c8871f7993
Parents: 90f0b66
Author: Hans Zeller <hz...@apache.org>
Authored: Tue Jan 9 22:10:33 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Tue Jan 9 22:10:33 2018 +0000

----------------------------------------------------------------------
 core/sql/udrserv/UdrResultSet.cpp | 22 +++++++++++-----------
 core/sql/udrserv/spinfo.cpp       | 14 +++++++-------
 core/sql/udrserv/udrserv.cpp      |  2 +-
 3 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/4d521af7/core/sql/udrserv/UdrResultSet.cpp
----------------------------------------------------------------------
diff --git a/core/sql/udrserv/UdrResultSet.cpp b/core/sql/udrserv/UdrResultSet.cpp
index 2c3388c..78beebc 100644
--- a/core/sql/udrserv/UdrResultSet.cpp
+++ b/core/sql/udrserv/UdrResultSet.cpp
@@ -545,7 +545,7 @@ UdrResultSet::setContext(SQLCTX_HANDLE &oldCtx, ComDiagsArea &d)
   Int32 result = SQL_EXEC_SwitchContext((Lng32) getContextHandle(),
                                       &tmpCtx);
 
-  if (result != 0)
+  if (result < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_SwitchContext")
@@ -567,7 +567,7 @@ UdrResultSet::resetContext(SQLCTX_HANDLE ctxHandle, ComDiagsArea &d)
   SQLCTX_HANDLE tmpCtxHandle;
   Int32 result = SQL_EXEC_SwitchContext(ctxHandle, &tmpCtxHandle);
 
-  if (result != 0)
+  if (result < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_SwitchContext")
@@ -634,7 +634,7 @@ UdrResultSet::generateProxySyntax(ComDiagsArea &d)
   outmodule->module_name =0;
 
   retcode = SQL_EXEC_AllocDesc(output_desc_, 500);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_AllocDesc")
@@ -645,7 +645,7 @@ UdrResultSet::generateProxySyntax(ComDiagsArea &d)
 
   // Describe Statement.
   retcode = SQL_EXEC_DescribeStmt(stmt_id_, NULL, output_desc_);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_DescribeStmt")
@@ -657,7 +657,7 @@ UdrResultSet::generateProxySyntax(ComDiagsArea &d)
   // Check how many columns there are in the output_desc
   ComUInt32 numColumns = 0;
   retcode = SQL_EXEC_GetDescEntryCount(output_desc_, (Lng32*) &numColumns);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_GetDescEntryCount")
@@ -751,7 +751,7 @@ UdrResultSet::generateProxySyntax(ComDiagsArea &d)
   retcode = SQL_EXEC_GetDescItems2(output_desc_,
                                    (Lng32) numColumns * NUMDESC_ITEMS,
                                    desc_items);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_GetDescItems2")
@@ -1252,7 +1252,7 @@ UdrResultSet::setupQuadFields(ComDiagsArea &d)
                                                1,
                                                (Lng32) numColumns_,
                                                quad_fields_);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_SETROWSETDESCPOINTERS")
@@ -1286,7 +1286,7 @@ UdrResultSet::fetchRowsFromCLI(UdrGlobals *udrGlob,
   if (stmt_id_->tag != 0)
   {
     retcode = SQL_EXEC_DisassocFileNumber(stmt_id_);
-    if (retcode != 0)
+    if (retcode < 0)
     {
       mainDiags << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
                 << DgString0("SQL_EXEC_DisassocFileNumber")
@@ -1321,7 +1321,7 @@ UdrResultSet::fetchRowsFromCLI(UdrGlobals *udrGlob,
          ! tmpBuffer_->moreRowsToCopy())
   {
     retcode = SQL_EXEC_ClearDiagnostics(stmt_id_);
-    if (retcode != 0)
+    if (retcode < 0)
     {
       mainDiags << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
                 << DgString0("SQL_EXEC_ClearDiagnostics")
@@ -1416,7 +1416,7 @@ UdrResultSet::fetchRowsFromCLI(UdrGlobals *udrGlob,
     }
 
     retcode = SQL_EXEC_ClearDiagnostics(stmt_id_);
-    if (retcode != 0)
+    if (retcode < 0)
     {
       mainDiags << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
                 << DgString0("SQL_EXEC_ClearDiagnostics")
@@ -1431,7 +1431,7 @@ UdrResultSet::fetchRowsFromCLI(UdrGlobals *udrGlob,
     retcode = SQL_EXEC_GetDescItem(output_desc_, 1,
                                    SQLDESC_ROWSET_NUM_PROCESSED,
                                    &numRowsFetched, 0, 0, 0, 0);
-    if (retcode != 0)
+    if (retcode < 0)
     {
       mainDiags << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
                 << DgString0("SQL_EXEC_GetDescItem")

http://git-wip-us.apache.org/repos/asf/trafodion/blob/4d521af7/core/sql/udrserv/spinfo.cpp
----------------------------------------------------------------------
diff --git a/core/sql/udrserv/spinfo.cpp b/core/sql/udrserv/spinfo.cpp
index 224592e..5de87c5 100644
--- a/core/sql/udrserv/spinfo.cpp
+++ b/core/sql/udrserv/spinfo.cpp
@@ -893,7 +893,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
   Lng32 retcode = 0;
 
   retcode = SQL_EXEC_ClearDiagnostics(NULL);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_ClearDiagnostics")
@@ -915,7 +915,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
   stmt->handle = 0;
 
   retcode = SQL_EXEC_AllocStmt(stmt, 0);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_AllocStmt")
@@ -938,7 +938,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
   sqlsrc_desc.identifier = 0;
   sqlsrc_desc.handle = 0;
   retcode = SQL_EXEC_AllocDesc(&sqlsrc_desc, 1);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_AllocDesc")
@@ -966,7 +966,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
   desc_items[2].num_val_or_len = (Lng32) strlen(sql_str) + 1;
 
   retcode = SQL_EXEC_SetDescItems2(&sqlsrc_desc, 3, desc_items);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     d << DgSqlCode(-UDR_ERR_INTERNAL_CLI_ERROR)
       << DgString0("SQL_EXEC_SetDescItem2")
@@ -982,7 +982,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
 
   // Prepare the statement; stmt has the prepared plan
   retcode = SQL_EXEC_Prepare(stmt, &sqlsrc_desc);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     SQL_EXEC_MergeDiagnostics_Internal(d);
 
@@ -1000,7 +1000,7 @@ SQLSTMT_ID *SPInfo::executeSqlStmt(const char *sql_str, ComDiagsArea &d)
 
   // Execute the statement
   retcode = SQL_EXEC_ExecClose(stmt, 0, 0, 0);
-  if (retcode != 0)
+  if (retcode < 0)
   {
     SQL_EXEC_MergeDiagnostics_Internal(d);
 
@@ -1285,7 +1285,7 @@ void SPInfo::quiesceExecutor()
                     "Message carried a transaction. About to quiesce.");
       
       Lng32 sqlcode = SQL_EXEC_Xact(SQLTRANS_QUIESCE, NULL);
-      if (sqlcode != 0)
+      if (sqlcode < 0)
       {
         char msg[MAXERRTEXT];
         str_sprintf(msg, "SQL_EXEC_Xact returned error %d", (Int32) sqlcode);

http://git-wip-us.apache.org/repos/asf/trafodion/blob/4d521af7/core/sql/udrserv/udrserv.cpp
----------------------------------------------------------------------
diff --git a/core/sql/udrserv/udrserv.cpp b/core/sql/udrserv/udrserv.cpp
index a118b1c..050898f 100644
--- a/core/sql/udrserv/udrserv.cpp
+++ b/core/sql/udrserv/udrserv.cpp
@@ -2105,7 +2105,7 @@ static Int32 invokeUdrMethod(const char *method,
     if (txRequired && result == LM_OK)
     {
       cliResult = SQL_EXEC_Xact(SQLTRANS_QUIESCE, NULL);
-      if (cliResult != 0)
+      if (cliResult < 0)
       {
         fprintf(f, "%s SQL_EXEC_Xact() returned %d\n",
                 prefix, cliResult);


[6/6] trafodion git commit: Merge [TRAFODION-2879] Core dump ...plus four more fixes, see below PR-1384

Posted by hz...@apache.org.
Merge [TRAFODION-2879] Core dump ...plus four more fixes, see below PR-1384


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

Branch: refs/heads/master
Commit: 72da2cd7d55860c2f81c5ee11717eff2da5b8987
Parents: fbd9185 dc94344
Author: Hans Zeller <hz...@apache.org>
Authored: Thu Jan 11 17:45:04 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Thu Jan 11 17:45:04 2018 +0000

----------------------------------------------------------------------
 core/sqf/sql/scripts/install_local_hadoop       |  2 +-
 core/sql/arkcmp/CmpContext.cpp                  |  3 --
 core/sql/arkcmp/CmpContext.h                    |  8 -----
 core/sql/arkcmp/CmpStatement.cpp                | 23 +++++++------
 core/sql/arkcmp/CmpStatement.h                  |  6 ++++
 core/sql/common/CmpCommon.h                     |  2 +-
 core/sql/optimizer/OptimizerSimulator.cpp       | 36 ++++++++++++++++----
 core/sql/optimizer/RelExpr.h                    |  1 +
 core/sql/regress/hive/EXPECTED005               |  4 +++
 core/sql/regress/hive/EXPECTED007               | 10 ++++--
 core/sql/regress/hive/EXPECTED009               | 12 +++----
 core/sql/regress/seabase/EXPECTED031            |  6 ++--
 core/sql/sqlcomp/CmpDescribe.cpp                | 10 +++---
 .../org/trafodion/sql/udr/predef/JDBCUDR.java   | 10 +++---
 core/sql/udrserv/UdrResultSet.cpp               | 22 ++++++------
 core/sql/udrserv/spinfo.cpp                     | 14 ++++----
 core/sql/udrserv/udrserv.cpp                    |  2 +-
 17 files changed, 101 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/72da2cd7/core/sql/arkcmp/CmpContext.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/72da2cd7/core/sql/arkcmp/CmpStatement.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafodion/blob/72da2cd7/core/sql/udrserv/udrserv.cpp
----------------------------------------------------------------------


[5/6] trafodion git commit: Updates to expected files (moved "Trafodion DDL" comment).

Posted by hz...@apache.org.
Updates to expected files (moved "Trafodion DDL" comment).


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

Branch: refs/heads/master
Commit: dc9434454f8868833849153d79191084b1bf6fcd
Parents: 40b1773
Author: Hans Zeller <hz...@apache.org>
Authored: Wed Jan 10 06:20:44 2018 +0000
Committer: Hans Zeller <hz...@apache.org>
Committed: Wed Jan 10 06:20:44 2018 +0000

----------------------------------------------------------------------
 core/sql/regress/hive/EXPECTED005    |  4 ++++
 core/sql/regress/hive/EXPECTED007    | 10 ++++++++--
 core/sql/regress/hive/EXPECTED009    | 12 ++++++------
 core/sql/regress/seabase/EXPECTED031 |  6 +++---
 4 files changed, 21 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafodion/blob/dc943445/core/sql/regress/hive/EXPECTED005
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/EXPECTED005 b/core/sql/regress/hive/EXPECTED005
index f91b49a..493fe4e 100644
--- a/core/sql/regress/hive/EXPECTED005
+++ b/core/sql/regress/hive/EXPECTED005
@@ -1002,6 +1002,8 @@ CREATE TABLE THIVE_INSERT_SMALLINT
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 --- SQL operation complete.
 >>
 >>truncate hive.hive.thive_insert_smallint;
@@ -1111,6 +1113,8 @@ CREATE TABLE THIVE_INSERT_VARCHAR
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 --- SQL operation complete.
 >>cqd hive_insert_error_mode '1';
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/dc943445/core/sql/regress/hive/EXPECTED007
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/EXPECTED007 b/core/sql/regress/hive/EXPECTED007
index 5bc2c3a..06d035b 100644
--- a/core/sql/regress/hive/EXPECTED007
+++ b/core/sql/regress/hive/EXPECTED007
@@ -47,6 +47,8 @@ CREATE TABLE THIVE1
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.THIVE1;
 /* ObjectUID = 7080927501909560103 */
 
@@ -257,6 +259,8 @@ CREATE TABLE THIVE2
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.THIVE2;
 /* ObjectUID = 1928809434068290686 */
 
@@ -1057,11 +1061,11 @@ CREATE TABLE THIVE9
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 REGISTER HIVE TABLE HIVE.HIVESCH007.THIVE9;
 /* ObjectUID = 6918234965366838662 */
 
-/* Trafodion DDL */
-
 CREATE EXTERNAL TABLE THIVE9
   FOR HIVE.HIVESCH007.THIVE9
 ;
@@ -1126,6 +1130,8 @@ CREATE TABLE THIVE9
   stored as textfile
 ;
 
+/* Trafodion DDL */
+
 --- SQL operation complete.
 >>showstats for table hive.hivesch007.thive9 on every column;
 

http://git-wip-us.apache.org/repos/asf/trafodion/blob/dc943445/core/sql/regress/hive/EXPECTED009
----------------------------------------------------------------------
diff --git a/core/sql/regress/hive/EXPECTED009 b/core/sql/regress/hive/EXPECTED009
index a5cdbf7..a59e5dd 100644
--- a/core/sql/regress/hive/EXPECTED009
+++ b/core/sql/regress/hive/EXPECTED009
@@ -999,11 +999,11 @@ CREATE TABLE DATE_DIM
   stored as textfile
 ;
 
-REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.DATE_DIM;
-/* ObjectUID = 1585406280815125826 */
-
 /* Trafodion DDL */
 
+REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.DATE_DIM;
+/* ObjectUID = 4145713645956211204 */
+
 CREATE EXTERNAL TABLE DATE_DIM
   (
     D_DATE_SK                        INT DEFAULT NULL
@@ -1233,11 +1233,11 @@ CREATE TABLE DATE_DIM
   stored as textfile
 ;
 
-REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.DATE_DIM;
-/* ObjectUID = 1585406280815125826 */
-
 /* Trafodion DDL */
 
+REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.DATE_DIM;
+/* ObjectUID = 4145713645956211204 */
+
 CREATE EXTERNAL TABLE DATE_DIM
   (
     D_DATE_SK                        INT DEFAULT NULL

http://git-wip-us.apache.org/repos/asf/trafodion/blob/dc943445/core/sql/regress/seabase/EXPECTED031
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED031 b/core/sql/regress/seabase/EXPECTED031
index 2619e1a..43a3d2d 100644
--- a/core/sql/regress/seabase/EXPECTED031
+++ b/core/sql/regress/seabase/EXPECTED031
@@ -1089,11 +1089,11 @@ CREATE TABLE T031HIVET1
   stored as textfile
 ;
 
-REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.T031HIVET1;
-/* ObjectUID = 6327691840910526408 */
-
 /* Trafodion DDL */
 
+REGISTER /*INTERNAL*/ HIVE TABLE HIVE.HIVE.T031HIVET1;
+/* ObjectUID = 3677902230547698142 */
+
 CREATE EXTERNAL TABLE T031HIVET1
   FOR HIVE.HIVE.T031HIVET1
 ;