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 2016/11/19 01:16:01 UTC

[1/2] incubator-trafodion git commit: [TRAFODION-2341] Redesign UPDATE STATISTICS retry logic

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master a596155f9 -> bbbd26ec8


[TRAFODION-2341] Redesign UPDATE STATISTICS retry logic


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

Branch: refs/heads/master
Commit: a39e85b742ba1ee3a412267ea4655dd49d22601d
Parents: 5162e5a
Author: Dave Birdsall <db...@apache.org>
Authored: Wed Nov 16 00:14:42 2016 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Wed Nov 16 00:14:42 2016 +0000

----------------------------------------------------------------------
 core/sql/ustat/hs_cli.cpp     | 361 +++++++++++++++++++++----------------
 core/sql/ustat/hs_cli.h       |  67 ++++++-
 core/sql/ustat/hs_globals.cpp | 121 +++++--------
 core/sql/ustat/hs_log.h       |  15 +-
 core/sql/ustat/hs_read.cpp    |   4 +-
 5 files changed, 328 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a39e85b7/core/sql/ustat/hs_cli.cpp
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_cli.cpp b/core/sql/ustat/hs_cli.cpp
index 69dbff8..77984f7 100644
--- a/core/sql/ustat/hs_cli.cpp
+++ b/core/sql/ustat/hs_cli.cpp
@@ -158,6 +158,7 @@ Lng32 HSExecDirect( SQLSTMT_ID * stmt
 }
 
 
+
 // -----------------------------------------------------------------------
 // DESCRIPTION: Execute a standalone dml/ddl statement.
 // INPUTS:  dml = text string of SQL query.
@@ -169,14 +170,15 @@ Lng32 HSExecDirect( SQLSTMT_ID * stmt
 //                executed (within ustats, ...).  Used for err processing.
 //          tabDef = pointer to HSTableDef of table affected (only used
 //                when rowsAffected, srcTabRowCount are non NULL.
-//          doRetry = if TRUE, retry the statement the configured (via CQD)
-//                number of times if it fails.
 //          errorToIgnore = sqlcode of an inconsequential expected error
 //                that should not disrupt execution, such as "schema already
 //                exists" when executing a Create Schema statement. 0 indicates
 //                there is no such expected error.
 //          checkMdam = if TRUE, determine whether the query uses MDAM, and
 //                include this information in the ulog.
+//          inactivateErrorCatcher = TRUE if the caller already has an
+//                HSErrorCatcher object active (that is, the caller wants
+//                to capture diagnostics itself).
 // -----------------------------------------------------------------------
 Lng32 HSFuncExecQuery( const char *dml
                     , short sqlcode
@@ -184,7 +186,34 @@ Lng32 HSFuncExecQuery( const char *dml
                     , const char *errorToken
                     , Int64 *srcTabRowCount
                     , const HSTableDef *tabDef
-                    , NABoolean doRetry
+                    , short errorToIgnore
+                    , NABoolean checkMdam
+                    , NABoolean inactivateErrorCatcher
+                    )
+{
+  Lng32 retcode;
+  // The HSErrorCatcher captures any diagnostics when it goes out-of-scope,
+  // unless it is inactivated (in which case it does nothing).
+  HSErrorCatcher errorCatcher(retcode, sqlcode, errorToken, TRUE,
+                              inactivateErrorCatcher);
+  retcode = HSFuncExecQueryBody(dml,sqlcode,rowsAffected,errorToken,
+                                srcTabRowCount,tabDef,errorToIgnore,checkMdam);
+  HSHandleError(retcode);
+  return retcode;
+}
+
+// -----------------------------------------------------------------------
+// DESCRIPTION: This is the body of HSFuncExecQuery. It is pulled out
+// as a separate function so it can also be used by 
+// HSFuncExecTransactionalQueryWithRetry. Each of the callers has its
+// own HSErrorCatcher object to capture diagnostics.
+// -----------------------------------------------------------------------
+Lng32 HSFuncExecQueryBody( const char *dml
+                    , short sqlcode
+                    , Int64 *rowsAffected
+                    , const char *errorToken
+                    , Int64 *srcTabRowCount
+                    , const HSTableDef *tabDef
                     , short errorToIgnore
                     , NABoolean checkMdam
                     )
@@ -201,8 +230,7 @@ Lng32 HSFuncExecQuery( const char *dml
   LM->Log(dml);
 
   Lng32 retcode;
-  HSErrorCatcher errorCatcher(retcode, sqlcode, errorToken, TRUE);
-
+ 
   SQLMODULE_ID module;
   SQLSTMT_ID stmt;
   SQLDESC_ID srcDesc;
@@ -274,68 +302,17 @@ Lng32 HSFuncExecQuery( const char *dml
             (char *)CmpCommon::context()->sqlSession()->getParentQid());
   HSHandleError(retcode);
 
-  if (!doRetry)
-  {
-    // execute immediate this statement
-    retcode = HSExecDirect(&stmt, &srcDesc, srcTabRowCount != 0, checkMdam);
-    // If retcode is > 0 or sqlcode is HS_WARNING, then set to 0 (no error/ignore).
-    if (retcode >= 0) retcode = 0;
-    // If sqlcode is HS_WARNING, then this means failures should be returned as
-    // warnings.  So, don't call HSHandleError, but rather return 0. Also return
-    // 0 if we get an expected and inconsequential error.
-    if ((sqlcode == HS_WARNING && retcode < 0) || retcode == errorToIgnore)
-      retcode = 0;
-    else
-      HSHandleError(retcode);
-  }
-  else // doRetry
-  {
-    // on very busy system, some "update statistics" implementation steps like
-    // "COLLECT FILE STATISTICS" step in HSTableDef::collectFileStatistics()
-    // that calls HSFuncExecQuery may experience transient failures that
-    // may succeed if retried enough times. We want to use AQR for these
-    // but AQR is not done for catapi, static queries, etc. For these, we
-    // are forced to do our own retry here.
-    Int32 centiSecs = getDefaultAsLong(USTAT_RETRY_DELAY);
-    Int32 limit = getDefaultAsLong(USTAT_RETRY_LIMIT);
-    for (Int32 retry = 0; retry <= limit; retry++) {
-
-      // clear the diags to get ready for the next execution attempt
-      if (retry > 0)
-      {
-         retcode = SQL_EXEC_ClearDiagnostics(&stmt);
-         HSHandleError(retcode);
-      }
-
-      // execute immediate this statement
-      retcode = HSExecDirect(&stmt, &srcDesc, srcTabRowCount != 0, checkMdam);
-
-      // filter retcode for HSHandleError
-      HSFilterWarning(retcode);
-      // If retcode is > 0 or sqlcode is HS_WARNING,
-      // then set to 0 (no error/ignore).
-      if (retcode >= 0) retcode = 0;
-      // If sqlcode is HS_WARNING, then failures should be ignored. Also check
-      // for specific error code to be ignored.
-      if ((sqlcode == HS_WARNING && retcode < 0) || retcode == errorToIgnore)
-        retcode = 0;
-
-      if (!retcode)
-        break; // passed ExecDirect
-      else if (retcode == -SQLCI_SYNTAX_ERROR)
-        break; // don't retry statements with syntax errors
-      else
-        {
-          ComDiagsArea diags(STMTHEAP);
-          SQL_EXEC_MergeDiagnostics_Internal(diags); // copy CLI diags area
-          if (diags.contains(-EXE_CANCELED))
-            break; // don't retry canceled query
-          else if (limit && retry < limit)
-            DELAY_CSEC(centiSecs); // failed & retry is on. so, wait & retry
-        }
-    }
-    if (sqlcode != HS_WARNING || retcode >= 0) HSHandleError(retcode);
-  }
+  // execute immediate this statement
+  retcode = HSExecDirect(&stmt, &srcDesc, srcTabRowCount != 0, checkMdam);
+  // If retcode is > 0 or sqlcode is HS_WARNING, then set to 0 (no error/ignore).
+  if (retcode >= 0) retcode = 0;
+  // If sqlcode is HS_WARNING, then this means failures should be returned as
+  // warnings.  So, don't call HSHandleError, but rather return 0. Also return
+  // 0 if we get an expected and inconsequential error.
+  if ((sqlcode == HS_WARNING && retcode < 0) || retcode == errorToIgnore)
+    retcode = 0;
+  else
+    HSHandleError(retcode);
 
   if (rowsAffected != NULL && tabDef != NULL)
     {
@@ -374,6 +351,127 @@ Lng32 HSFuncExecQuery( const char *dml
   return retcode;
 }
 
+// -----------------------------------------------------------------------
+// DESCRIPTION: Execute a standalone dml/ddl statement within a 
+//              locally-started and committed transaction, with retry.
+//              Many Trafodion errors abort the transaction; in order
+//              to retry statements we need to manage the transaction
+//              here as well.
+// INPUTS:  dml = text string of SQL query.
+//          sqlcode = the error to issue upon failure, or HS_WARNING if
+//                errors should be suppressed.
+//          rowsAffected, srcTabRowCount  = pointers (NULL by default) to
+//                variables that rowcount info for query will be stored in.
+//          errorToken = text string indicating major operation being
+//                executed (within ustats, ...).  Used for err processing.
+//          tabDef = pointer to HSTableDef of table affected (only used
+//                when rowsAffected, srcTabRowCount are non NULL.
+//          errorToIgnore = sqlcode of an inconsequential expected error
+//                that should not disrupt execution, such as "schema already
+//                exists" when executing a Create Schema statement. 0 indicates
+//                there is no such expected error.
+//          checkMdam = if TRUE, determine whether the query uses MDAM, and
+//                include this information in the ulog.
+// -----------------------------------------------------------------------
+Lng32 HSFuncExecTransactionalQueryWithRetry( const char *dml
+                                           , short sqlcode
+                                           , Int64 *rowsAffected
+                                           , const char *errorToken
+                                           , Int64 *srcTabRowCount
+                                           , const HSTableDef *tabDef
+                                           , short errorToIgnore
+                                           , NABoolean checkMdam
+                                           )
+{
+  HSLogMan *LM = HSLogMan::Instance();
+  HSTranMan *TM = HSTranMan::Instance();
+  HSGlobalsClass *hs_globals = GetHSContext();
+  Lng32 retcode = 0;
+
+  if (TM->InTransaction())
+  {
+    // a transaction is already in progress; can't do retry
+    // as we don't know if there is other work already in the
+    // transaction
+    *CmpCommon::diags() << DgSqlCode(-UERR_GENERIC_ERROR)
+                        << DgString0("HSFuncExecTransactionalQueryWithRetry")
+                        << DgString1("-9215")
+                        << DgString2("Unexpected transaction in progress");
+    retcode = -UERR_GENERIC_ERROR;
+    return retcode;
+  }
+
+  // The HSErrorCatcher captures any diagnostics when it goes out-of-scope.
+  HSErrorCatcher errorCatcher(retcode, sqlcode, errorToken, TRUE);
+ 
+  // On very busy system, some "update statistics" implementation steps like
+  // "COLLECT FILE STATISTICS" step in HSTableDef::collectFileStatistics()
+  // may experience transient failures that may succeed if retried enough
+  // times. Note that AQR may do some retries for us under the covers.
+  Int32 centiSecs = getDefaultAsLong(USTAT_RETRY_DELAY);
+  Int32 limit = getDefaultAsLong(USTAT_RETRY_LIMIT);
+  for (Int32 retry = 0; retry <= limit; retry++)
+  {  
+    // start a transaction
+    retcode = TM->Begin("Transaction for retryable Statement",TRUE);
+    if (retcode == 0)
+    {
+      // execute the statement
+ 
+      retcode = HSFuncExecQueryBody(dml, sqlcode, rowsAffected, errorToken,
+                                srcTabRowCount, tabDef, errorToIgnore, checkMdam);
+
+      // Figure out if we want to ignore certain conditions 
+
+      // filter retcode for HSHandleError
+      HSFilterWarning(retcode);
+      // If retcode is > 0 then set to 0 (no error/ignore).
+      if (retcode > 0)
+        retcode = 0;
+      // If sqlcode is HS_WARNING, then failures should be ignored. Also check
+      // for specific error code to be ignored.
+      if ((sqlcode == HS_WARNING && retcode < 0) || retcode == errorToIgnore)
+        retcode = 0;
+
+      // if successful, commit the transaction
+      if (retcode == 0)
+        retcode = TM->Commit(TRUE);
+
+      // analyze any error from the statement or the commit to see
+      // if we should retry
+      if (retcode == 0)
+        retry = limit+1; // exit retry loop on success
+      if (retcode == -SQLCI_SYNTAX_ERROR)
+        retry = limit+1; // don't retry statements with syntax errors
+      else if (retcode)
+      {
+        ComDiagsArea diags(STMTHEAP);
+        SQL_EXEC_MergeDiagnostics_Internal(diags); // copy CLI diags area
+        if (diags.contains(-EXE_CANCELED))
+          retry = limit+1; // don't retry canceled query
+      }
+    }
+
+    // If we had an error (on the begin, the statement or the commit),
+    // try rolling back the transaction. It might have already been
+    // rolled back, in which case the Rollback method just ignores
+    // the error.
+    if (retcode)
+    {
+      TM->Rollback(TRUE);
+      if (retry < limit)  // if there are more retries
+        DELAY_CSEC(centiSecs); // wait before retrying
+    }
+  }
+    
+  if (sqlcode != HS_WARNING || retcode < 0) 
+    HSHandleError(retcode);
+
+  return retcode;
+}
+
+
+
 Lng32 HSFuncExecDDL( const char *dml
                   , short sqlcode
                   , Int64 *rowsAffected
@@ -386,35 +484,9 @@ Lng32 HSFuncExecDDL( const char *dml
   if (!tabDef && hs_globals) tabDef = hs_globals->objDef;
   if (!tabDef) return -1;
 
-  // Whenever a DDL request is made we must always start a transaction prior 
-  // to the request. If we do not, in certain cases, a transaction will be 
-  // started and never committed. See LP bug 1404442 
-  HSTranMan *TM;
-  NABoolean startedTrans = FALSE;
-  TM = HSTranMan::Instance();
-  {
-    // This HSErrorCatcher is in its own block so we'll report a
-    // diagnostic in case the TM->Begin call fails. Note that
-    // HSFuncExecQuery contains its own HSErrorCatcher, so that 
-    // code is outside this block. (If we included it in this
-    // block then any error encountered there would be reported
-    // twice.)
-    HSErrorCatcher errorCatcher(retcode, sqlcode, errorToken, TRUE);
-    startedTrans = (((retcode = TM->Begin("DDL")) == 0) ? TRUE : FALSE);    
-    HSHandleError(retcode);
-  }
-
-  //Special parser flags needed to use the NO AUDIT option.
-  retcode = HSFuncExecQuery(dml, sqlcode, rowsAffected, errorToken,
-                            NULL, tabDef, TRUE/*doRetry*/);
+  retcode = HSFuncExecTransactionalQueryWithRetry(dml, sqlcode, 
+                             rowsAffected, errorToken, NULL, tabDef);
 
-  if (startedTrans)
-    {
-      if (retcode)
-        TM->Rollback();
-      else
-        TM->Commit();
-    }
   HSHandleError(retcode);
 
   return retcode;
@@ -940,10 +1012,14 @@ HSTranMan* HSTranMan::Instance()
 /*                4 - transaction is running   */
 /*          SQLCODE - severe error             */
 /***********************************************/
-Lng32 HSTranMan::Begin(const char *title)
+Lng32 HSTranMan::Begin(const char *title,NABoolean inactivateErrorCatcher)
   {
     HSLogMan *LM = HSLogMan::Instance();
 
+    // if an error occurred on a previous HSTransMan call, try to clean it up here
+    if ((retcode_ < 0) && (!InTransaction()))
+      retcode_ = 0;  // not in a transaction, so OK to try starting one
+
     if (retcode_ < 0)                              /*== ERROR HAD OCCURRED ==*/
       {
         if (LM->LogNeeded())
@@ -960,7 +1036,7 @@ Lng32 HSTranMan::Begin(const char *title)
           {
             NAString stmtText = "BEGIN WORK";
             retcode_ = HSFuncExecQuery(stmtText.data(), - UERR_INTERNAL_ERROR, NULL,
-                                       HS_QUERY_ERROR, NULL, NULL, TRUE);
+                                       HS_QUERY_ERROR, NULL, NULL, inactivateErrorCatcher);
             if (retcode_ >= 0)
               {
                 transStarted_ = TRUE;
@@ -997,7 +1073,7 @@ Lng32 HSTranMan::Begin(const char *title)
 /*                    transaction              */
 /*          SQLCODE - severe error             */
 /***********************************************/
-Lng32 HSTranMan::Commit()
+Lng32 HSTranMan::Commit(NABoolean inactivateErrorCatcher)
   {
     HSLogMan *LM = HSLogMan::Instance();
 
@@ -1015,7 +1091,7 @@ Lng32 HSTranMan::Commit()
           {
             NAString stmtText = "COMMIT WORK";
             retcode_ = HSFuncExecQuery(stmtText.data(), - UERR_INTERNAL_ERROR, NULL,
-                                       HS_QUERY_ERROR, NULL, NULL, TRUE);
+                                       HS_QUERY_ERROR, NULL, NULL, inactivateErrorCatcher);
 
             // transaction has ended
             transStarted_ = FALSE;
@@ -1047,9 +1123,12 @@ void HSTranMan::logXactCode(const char* title)
     if (LM->LogNeeded()) {
       LM->Log(title);
       Lng32 transCode = SQL_EXEC_Xact(SQLTRANS_STATUS, 0);
-      char buf[50];
+      char buf[80];
       snprintf(buf, sizeof(buf), "SQL_EXEC_Xact() code=%d\n", transCode);
       LM->Log(buf);
+      snprintf(buf, sizeof(buf), "transStarted_ = %s, extTrans_ = %s, retcode_ = %d\n",
+        transStarted_ ? "TRUE" : "FALSE", extTrans_ ? "TRUE" : "FALSE", retcode_);
+      LM->Log(buf);   
     }
  }
 
@@ -1064,10 +1143,22 @@ void HSTranMan::logXactCode(const char* title)
 /*                    transaction              */
 /*          SQLCODE - severe error             */
 /***********************************************/
-Lng32 HSTranMan::Rollback()
+Lng32 HSTranMan::Rollback(NABoolean inactivateErrorCatcher)
   {
     HSLogMan *LM = HSLogMan::Instance();
 
+    // if an error occurred on a previous HSTransMan call, try to clean it up here
+    if ((retcode_ < 0) && (!InTransaction()))
+      {
+        retcode_ = 0;  // not in a transaction, and therefore no need to roll back
+        if (LM->LogNeeded())
+          {
+            snprintf(LM->msg, sizeof(LM->msg), "ROBACKWORK(cleaned up; not in transaction)");
+            LM->Log(LM->msg);
+          }       
+        return retcode_;
+      }
+
     if (retcode_ < 0)                              /*== ERROR HAD OCCURRED ==*/
       {
         if (LM->LogNeeded())
@@ -1082,7 +1173,7 @@ Lng32 HSTranMan::Rollback()
           {
             NAString stmtText = "ROLLBACK WORK";
             retcode_ = HSFuncExecQuery(stmtText.data(), - UERR_INTERNAL_ERROR, NULL,
-                                       HS_QUERY_ERROR, NULL, NULL, TRUE);
+                                       HS_QUERY_ERROR, NULL, NULL, inactivateErrorCatcher);
             // transaction has ended
             transStarted_ = FALSE;
             if (retcode_ < 0)
@@ -1218,29 +1309,6 @@ void HSTranController::stopStart(const char* title)
     logMan_->LogTimestamp("Transaction started");
 }
 
-/**************************************************/
-/* METHOD:  HSTranController::lockTable           */
-/* PURPOSE: Static function that locks the given  */
-/*          table in either shared or exclusive   */
-/*          mode, as determined by the 'exclusive'*/
-/*          parameter.                            */
-/* RETCODE: 0 if no errors.                       */
-/*          non-zero otherwise.                   */
-/**************************************************/
-Lng32 HSTranController::lockTable(const char* tableName, NABoolean exclusive)
-{
-  NAString lockStmt("lock table ");
-  lockStmt += tableName;
-  if (exclusive)
-    lockStmt += " in exclusive mode";
-  else
-    lockStmt += " in shared mode";
-
-  return HSFuncExecQuery
-    (lockStmt.data(), - UERR_INTERNAL_ERROR, NULL,
-     HS_QUERY_ERROR, NULL, NULL, TRUE/*doRetry*/ );
-}
-
 
 /*****************************************************************************/
 /* CLASS:   HSPersSamples                                                    */
@@ -1443,6 +1511,11 @@ Lng32 HSPersSamples::find(HSTableDef *objDef, Int64 &actualRows, NABoolean isEst
       findSampleTableCursor1.close();
     else
       retcode = findSampleTableCursor1.close();
+
+    HSTranMan * TM = HSTranMan::Instance();
+    if (TM->InTransaction())
+      TM->Commit();
+
     return retcode;
   }
 
@@ -1551,7 +1624,8 @@ Lng32 HSPersSamples::find(HSTableDef *objDef, char reason,
       findSampleTableCursor.close();
     else
       retcode = findSampleTableCursor.close();
-   return retcode;
+
+    return retcode;
   }
 
 Lng32 HSPersSamples::removeSample(HSTableDef* tabDef, NAString& sampTblName,
@@ -1562,8 +1636,6 @@ Lng32 HSPersSamples::removeSample(HSTableDef* tabDef, NAString& sampTblName,
 
   if (sampTblName.length() > 0)
     {
-      TM->Begin(txnLabel);
-
       // Delete row in persistent samples table regardless of whether sample
       // could be dropped.
       ComObjectName persSampTblObjName(*catalog_,
@@ -1582,17 +1654,10 @@ Lng32 HSPersSamples::removeSample(HSTableDef* tabDef, NAString& sampTblName,
       dml += Int64ToNAString(objUID);
       // for now, the reason is ignored
 
-      retcode = HSFuncExecQuery(dml, - UERR_INTERNAL_ERROR, NULL,
-                                HS_QUERY_ERROR, NULL, NULL, TRUE/*doRetry*/ );
-      HSHandleError(retcode);
-
+      retcode = HSFuncExecTransactionalQueryWithRetry(dml, - UERR_INTERNAL_ERROR,
+                                 NULL, txnLabel, NULL, NULL);
       HSSample::dropSample(sampTblName, tabDef);
-
-      if (retcode)
-        TM->Rollback();
-      else
-        TM->Commit();
-
+      HSHandleError(retcode);
     }
 
   return retcode;
@@ -1667,6 +1732,7 @@ Lng32 HSPersSamples::createAndInsert(HSTableDef *tabDef, NAString &sampleName,
                          );
       // sampleName output & actualRows will get modified if necessary
       //  (based on isEstimate).
+
     if (!retcode)
     {
 
@@ -1683,7 +1749,6 @@ Lng32 HSPersSamples::createAndInsert(HSTableDef *tabDef, NAString &sampleName,
                                        STMTHEAP);
       into_table = persSampTblObjName.getExternalName();
       objUID = tabDef->getObjectUID();
-      TM->Begin("INSERT INTO SB_PERSISTENT_SAMPLES TABLE.");
       char timeStr[HS_TIMESTAMP_SIZE];
       hs_formatTimestamp(timeStr);
 
@@ -1721,15 +1786,13 @@ Lng32 HSPersSamples::createAndInsert(HSTableDef *tabDef, NAString &sampleName,
       dml += ",_UCS2''";  // V2
       dml += ");";
 
-      retcode = HSFuncExecQuery(dml, - UERR_INTERNAL_ERROR, NULL,
-                                HS_QUERY_ERROR, NULL, NULL, TRUE/*doRetry*/ );
-      HSHandleError(retcode);
-
-      if (!retcode) TM->Commit();
-      else {
-        TM->Rollback();
+      retcode = HSFuncExecTransactionalQueryWithRetry(dml, - UERR_INTERNAL_ERROR,
+                                 NULL, HS_QUERY_ERROR, NULL, NULL);
+      HSFilterWarning(retcode);  // can't do HSHandleError here since we want to do the drop    
+      if (retcode)
         sample.drop();
-      }
+      
+      HSHandleError(retcode);
     }
 
     return retcode;
@@ -1763,12 +1826,10 @@ Lng32 HSPersSamples::removeMatchingSamples(HSTableDef *tabDef,
                                            Int64 sampleRows,
                                            double allowedDiff)
   {
-    HSTranMan *TM = HSTranMan::Instance();
     Lng32 retcode = 0;
     NAString ddl, table, fromTable;
     NABoolean nothingToDrop = TRUE;
 
-    TM->Begin("DROP PERSISTENT SAMPLE TABLE AND REMOVE FROM LIST.");
     // Loop until all persistent samples matching criteria have been removed.
     Int64 actualRows = -1; // Obsolete samples will not be removed by find().
     NABoolean isEstimate = TRUE;
@@ -1788,8 +1849,6 @@ Lng32 HSPersSamples::removeMatchingSamples(HSTableDef *tabDef,
       if (!retcode)
         retcode = find(tabDef, actualRows, isEstimate, sampleRows, allowedDiff, table);
     }
-    if (retcode) TM->Rollback();
-    else         retcode = TM->Commit();
 
     if (nothingToDrop)
     {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a39e85b7/core/sql/ustat/hs_cli.h
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_cli.h b/core/sql/ustat/hs_cli.h
index e30a102..b3bf1f9 100644
--- a/core/sql/ustat/hs_cli.h
+++ b/core/sql/ustat/hs_cli.h
@@ -67,14 +67,56 @@ template <class T> class HSPtrArray;
 // -----------------------------------------------------------------------
 // Functions.
 // -----------------------------------------------------------------------
-// Execute a standalone dml/ddl operation.
+// Execute a standalone dml/ddl operation, without retry.
 Lng32 HSFuncExecQuery( const char *dml
                     , short sqlcode = - UERR_INTERNAL_ERROR
                     , Int64 *rowsAffected = NULL
                     , const char *errorToken = HS_QUERY_ERROR
                     , Int64 *srcTabRowCount = NULL
                     , const HSTableDef *tabDef = NULL
-                    , NABoolean doRetry = FALSE
+                    , short errorToIgnore = 0
+                    , NABoolean checkMdam = FALSE
+                    , NABoolean inactivateErrorCatcher = FALSE
+                    );
+
+// Body shared between HSFuncExecQuery and 
+// HSFuncExecTransactionalQueryWithRetry
+Lng32 HSFuncExecQueryBody( const char *dml
+                    , short sqlcode
+                    , Int64 *rowsAffected
+                    , const char *errorToken
+                    , Int64 *srcTabRowCount
+                    , const HSTableDef *tabDef
+                    , short errorToIgnore
+                    , NABoolean checkMdam
+                    );
+
+// Execute a standalone dml/ddl operation, with retry. Note
+// that this method handles starting and committing or rolling
+// back the transaction (as that needs to be part of the retry
+// loop since Trafodion often aborts transactions when
+// statements fail). Therefore, this function cannot be called
+// within a transaction. In fact it guards against this and will
+// return an error if it is called within a transaction. One
+// consequence is that calls to this function cannot be
+// done within the scope of an HSTranController object.
+//
+// Some thought should go into the choice of whether to use
+// this function vs. HSFuncExecQuery. It is appropriate to use
+// this function if the effect on the database is idempotent.
+// Examples of this are most DDL operations, DELETE, UPDATE
+// and INSERT statements that don't depend on prior state and
+// don't include non-deterministic semantics such as SAMPLE.
+// Operations that have non-transactional effects and non-
+// deterministic semantics (such as UPSERT USING LOAD with a
+// SAMPLE clause) should not be retried at this level.
+//
+Lng32 HSFuncExecTransactionalQueryWithRetry( const char *dml
+                    , short sqlcode = - UERR_INTERNAL_ERROR
+                    , Int64 *rowsAffected = NULL
+                    , const char *errorToken = HS_QUERY_ERROR
+                    , Int64 *srcTabRowCount = NULL
+                    , const HSTableDef *tabDef = NULL
                     , short errorToIgnore = 0
                     , NABoolean checkMdam = FALSE
                     );
@@ -310,9 +352,13 @@ class HSTranMan
   {
     public:
          static HSTranMan* Instance();
-         Lng32 Begin(const char *title = ""); /* Begin Transaction            */
-         Lng32 Commit();                   /* Commit Transaction              */
-         Lng32 Rollback();                 /* Rollback Transaction            */
+         /* Begin Transaction               */
+         Lng32 Begin(const char *title = "",
+                     NABoolean inactivateErrorCatcher=FALSE);
+         /* Commit Transaction              */
+         Lng32 Commit(NABoolean inactivateErrorCatcher=FALSE);
+         /* Rollback Transaction            */
+         Lng32 Rollback(NABoolean inactivateErrorCatcher=FALSE);
 
          // This method will tell you if there is currently a transaction
          // running. The transaction could have been started by USER or this
@@ -382,6 +428,14 @@ class HSPrologEpilog
 /* The decision as to whether the transaction should be committed or rolled  */
 /* back in the dtor is determined by the value of a return code, the address */
 /* of which is passed to HSTranController's ctor.                            */
+/*                                                                           */
+/* Limitations: Unfortunately, Trafodion often aborts transactions when DDL  */
+/* or DML statements fail. This precludes using HSTranController in retry    */
+/* scenarios as the retry would either fail due to lack of a transaction or  */
+/* (worse) succeed in a separate transaction. The latter is worse because    */
+/* any other work done in the original transaction would silently be undone. */
+/* To guard against this, HSFuncExecTransactionalQueryWithRetry raises an    */
+/* error if done within the scope of an HSTranController object.             */
 /*****************************************************************************/
 class HSTranController
   {
@@ -392,9 +446,6 @@ class HSTranController
       // Function to stop current transaction and start a new one.
       void stopStart(const char* title);
 
-      // Static function to acquire a lock on a table.
-      static Lng32 lockTable(const char* tableName, NABoolean exclusive = TRUE);
-
     private:
       HSTranMan* tranMan_;
       HSLogMan*  logMan_;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a39e85b7/core/sql/ustat/hs_globals.cpp
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_globals.cpp b/core/sql/ustat/hs_globals.cpp
index eedfe67..406287d 100644
--- a/core/sql/ustat/hs_globals.cpp
+++ b/core/sql/ustat/hs_globals.cpp
@@ -6448,56 +6448,25 @@ Lng32 HSGlobalsClass::prepareForIUSAlgorithm1(Int64& rows)
 
   // Update the sample table in two separate transactions.
   Int64 xRows;
-  
-  Int64 transId=-1;
-
-  // set inTranx to TRUE to avoid emit WITH NO ROLLBACK clause for -D and +I.
-  // It was found that the 2nd compiler and the executor contained in 
-  // 1st compiler do not agree on transaction attributes, refer to . 
-  // compareTransModes() in cli/Statement.cpp.
-  // 
-// >>update statistics for table tbl on existing column incremental where a>900;
-
-// *** ERROR[9200] UPDATE STATISTICS for table NEO.USR.TBL encountered an error (8814) from statement IUS S(i-1)-D+I operation.
-
-// *** WARNING[8814] The transaction mode at run time (300121) differs from that specified at compile time (100140).
-
-// *** ERROR[8814] The transaction mode at run time (300121) differs from that specified at compile time (100140).
-
-// *** ERROR[8839] Transaction was aborted.
-
-  //
-  // The method GenericUpdate::bindNode() in BinreRelExpr.cpp assumes
-  // that the compiler knows about transaction and NO ROLLBACK has to be
-  // bound when there is NO transaction. But the begin/commit work used by
-  // update stats are static SQL and their effort is not recognized by the
-  // 2nd compiler!
-  // 
  
-  { // first delete the old rows 
+  // first delete the old rows 
     
-    HSTranController TC("IUS: Update S with D", &retcode);
-    HSHandleError(retcode);
+  HSHandleError(retcode);
   
-    // temp. for now to generate the delQuery, set #rows to 1
-    iusSampleDeletedInMem = new(STMTHEAP) 
+  // temp. for now to generate the delQuery, set #rows to 1
+  iusSampleDeletedInMem = new(STMTHEAP) 
                               HSInMemoryTable(*hssample_table,
                                                getWherePredicateForIUS(),
                                                1 // rows
                                               );
   
-    // Populate the deleted rows into a in-memory table.
-    NAString delQuery;
-    generateIUSDeleteQuery(*hssample_table, delQuery);
-  
-  
+  NAString delQuery;
+  generateIUSDeleteQuery(*hssample_table, delQuery);
   
-    retcode = HSFuncExecQuery(delQuery, -UERR_INTERNAL_ERROR, &xRows, 
-                              "IUS S(i-1)-D operation",
-                               NULL, NULL, TRUE/*doRetry*/ );
-    HSHandleError(retcode);
-  
-  }
+  retcode = HSFuncExecTransactionalQueryWithRetry(delQuery, -UERR_INTERNAL_ERROR, 
+                              &xRows,"IUS S(i-1)-D operation",
+                              NULL, NULL);
+  HSHandleError(retcode);
 
   if (LM->LogNeeded())
      LM->StopTimer();
@@ -6530,13 +6499,12 @@ Lng32 HSGlobalsClass::prepareForIUSAlgorithm1(Int64& rows)
                                                sampleRateAsPercetageForIUS);
   
     NAString insQuery;
-    iusSampleInsertedInMem->generateInsertQuery(*hssample_table, *user_table, insQuery, FALSE);
+    iusSampleInsertedInMem->generateInsertQuery(*hssample_table, *user_table, insQuery, FALSE);  
   
-  
-    //retcode = iusSampleInsertedInMem->populate(insQuery);
-    retcode = HSFuncExecQuery(insQuery, -UERR_INTERNAL_ERROR, &xRows, 
-                              "IUS S(i-1)-D+I operation",
-                               NULL, NULL, TRUE/*doRetry*/ );
+    // Note that we don't retry the insert
+    retcode = HSFuncExecQuery(insQuery, -UERR_INTERNAL_ERROR, 
+                               &xRows, "IUS S(i-1)-D+I operation",
+                               NULL, NULL);
     HSHandleError(retcode);
   }
   
@@ -6568,9 +6536,9 @@ static Lng32 create_I(NAString& sampTblName)
   createI += "_I LIKE ";
   createI += sampTblName;
   createI += " WITH PARTITIONS";
-  Lng32 retcode = HSFuncExecQuery(createI, -UERR_INTERNAL_ERROR,
+  Lng32 retcode = HSFuncExecTransactionalQueryWithRetry(createI, -UERR_INTERNAL_ERROR,
                                   NULL, "IUS create I",
-                                  NULL, NULL, TRUE/*doRetry*/);
+                                  NULL, NULL);
   if (LM->LogNeeded())
      LM->StopTimer();
 
@@ -6586,8 +6554,6 @@ Lng32 HSGlobalsClass::generateSampleI(Int64 currentSampleSize,
   Lng32 retcode = 0;
   Int64 xRows;
 
-  Int64 transId=-1;
-
   HSLogMan *LM = HSLogMan::Instance();
   if (LM->LogNeeded())
      LM->StartTimer("IUS: select-insert data set I");
@@ -6611,9 +6577,11 @@ Lng32 HSGlobalsClass::generateSampleI(Int64 currentSampleSize,
                         deleteSetSize, actualRowCount);
 
     NABoolean needEspParReset = setEspParallelism(objDef);
-    retcode = HSFuncExecQuery(insertSelectIQuery, -UERR_INTERNAL_ERROR, &xRows,
+    // note that we can't do a retry on non-transactional upsert using load + sample
+    retcode = HSFuncExecQuery(insertSelectIQuery, 
+                              -UERR_INTERNAL_ERROR, &xRows,
                               "IUS data set I creation",
-                              NULL, NULL, TRUE/*doRetry*/,
+                              NULL, NULL,
                               0, TRUE);  // check for MDAM usage
 
     if (needEspParReset)
@@ -6643,9 +6611,9 @@ static Lng32 drop_I(NAString& sampTblName)
 
   NAString cleanupI("drop table if exists ");
   cleanupI.append(sampTblName).append("_I");
-  Lng32 retcode = HSFuncExecQuery(cleanupI, -UERR_INTERNAL_ERROR,
+  Lng32 retcode = HSFuncExecTransactionalQueryWithRetry(cleanupI, -UERR_INTERNAL_ERROR,
                                   NULL, "IUS cleanup I",
-                                  NULL, NULL, TRUE/*doRetry*/);
+                                  NULL, NULL);
   if (LM->LogNeeded())
     LM->StopTimer();
   HSHandleError(retcode);
@@ -6829,7 +6797,6 @@ Lng32 HSGlobalsClass::UpdateIUSPersistentSampleTable(Int64 oldSampleSize,
 
   HSFuncExecQuery("CONTROL QUERY DEFAULT ALLOW_DML_ON_NONAUDITED_TABLE 'ON'");
 
-  HSTranController TC("IUS: update PS table", &retcode);
   HSHandleError(retcode);
 
   // step 1  - delete the affected rows from PS
@@ -6843,10 +6810,10 @@ Lng32 HSGlobalsClass::UpdateIUSPersistentSampleTable(Int64 oldSampleSize,
   }
 
   rowsAffected = 0;
-  retcode = HSFuncExecQuery(deleteQuery, -UERR_INTERNAL_ERROR,
+  retcode = HSFuncExecTransactionalQueryWithRetry(deleteQuery, -UERR_INTERNAL_ERROR,
                             &rowsAffected,
                             "IUS delete from PS where",
-                            NULL, NULL, TRUE/*doRetry*/ );
+                            NULL, NULL);
   if (LM->LogNeeded()) {
     LM->StopTimer();
     sprintf(LM->msg, PF64 " rows deleted from persistent sample table.", rowsAffected);
@@ -6869,10 +6836,14 @@ Lng32 HSGlobalsClass::UpdateIUSPersistentSampleTable(Int64 oldSampleSize,
   rowsAffected = 0;
   const char* insSourceTblName = extractTblName(*hssample_table + "_I", objDef);
   NABoolean needEspParReset = setEspParallelism(objDef, insSourceTblName);
+ 
+  // can't retry this one, as it uses non-transactional upsert using load + random
+  // select; a retry might add *another* random sample to a partial sample from
+  // the previous attempt
   retcode = HSFuncExecQuery(selectInsertQuery, -UERR_INTERNAL_ERROR,
                             &rowsAffected,
                             "IUS insert into PS (select from _I)",
-                            NULL, NULL, TRUE/*doRetry*/, 0,
+                            NULL, NULL, 0,
                             // check mdam usage if reading incremental sample directly from source table
                             CmpCommon::getDefault(USTAT_INCREMENTAL_UPDATE_STATISTICS) == DF_SAMPLE);  //checkMdam
   if (LM->LogNeeded()) {
@@ -7728,18 +7699,6 @@ Lng32 HSGlobalsClass::FlushStatistics(NABoolean &statsWritten)
           HSTranController TC("FLUSH STATISTICS", &retcode);
           HSHandleError(retcode);
 
-          if (CmpCommon::getDefault(USTAT_LOCK_HIST_TABLES) == DF_ON)
-          {
-            // Lock hist tables to assure no deadlock if a concurrent transaction
-            // is updating stats on the same table.
-            retcode = HSTranController::lockTable(*(GetHSContext()->hstogram_table));
-            HSHandleError(retcode);
-            retcode = HSTranController::lockTable(*(GetHSContext()->hsintval_table));
-            HSHandleError(retcode);
-            retcode = HSTranController::lockTable(*hssample_table);
-            HSHandleError(retcode);
-          }
-
           // IUS work: Keep warnings unless there are errors.
           if ( CmpCommon::diags()->getNumber(DgSqlCode::ERROR_) > 0 )
               CmpCommon::diags()->clear();
@@ -8149,8 +8108,10 @@ Lng32 HSGlobalsClass::removeHists(NAString &hists, char *uid, const char *operat
       stmt += " AND HISTOGRAM_ID IN (";
       stmt += hists;
       stmt += ")";
-      retcode = HSFuncExecQuery(stmt, -UERR_INTERNAL_ERROR, &xRows, operation,
-                                NULL, NULL, TRUE/*doRetry*/ );
+      // Note that this can't be done with retry since we are
+      // part of a larger transaction started by FlushStatistics
+      retcode = HSFuncExecQuery(stmt, -UERR_INTERNAL_ERROR,
+                                 &xRows, operation, NULL, NULL);
       LM->StopTimer();
       HSHandleError(retcode);
       if (LM->LogNeeded())
@@ -8168,8 +8129,10 @@ Lng32 HSGlobalsClass::removeHists(NAString &hists, char *uid, const char *operat
       stmt += " AND HISTOGRAM_ID IN (";
       stmt += hists;
       stmt += ")";
-      retcode = HSFuncExecQuery(stmt, -UERR_INTERNAL_ERROR, &xRows, operation,
-                                NULL, NULL, TRUE/*doRetry*/ );
+      // Note that this can't be done with retry since we are
+      // part of a larger transaction started by FlushStatistics
+      retcode = HSFuncExecQuery(stmt, -UERR_INTERNAL_ERROR, 
+                                &xRows, operation, NULL, NULL);
       LM->StopTimer();
       HSHandleError(retcode);
       if (LM->LogNeeded())
@@ -9890,9 +9853,9 @@ Lng32 HSGlobalsClass::DeleteOrphanHistograms()
         query += " WHERE TABLE_UID NOT IN (SELECT CREATETIME FROM ";
         query += objDef->getCatalogLoc(HSTableDef::EXTERNAL_FORMAT);
         query += ".TABLES)";
-        retcode = HSFuncExecQuery
+        retcode = HSFuncExecTransactionalQueryWithRetry
           (query, -UERR_INTERNAL_ERROR, &rows, "CLEAR_ORPHANS",
-           NULL, NULL, TRUE/*doRetry*/ );
+           NULL, NULL);
         if (LM->LogNeeded())
           {
             convertInt64ToAscii(rows, rowCountStr);
@@ -9905,9 +9868,9 @@ Lng32 HSGlobalsClass::DeleteOrphanHistograms()
         query += " WHERE TABLE_UID NOT IN (SELECT CREATETIME FROM ";
         query += objDef->getCatalogLoc(HSTableDef::EXTERNAL_FORMAT);
         query += ".TABLES)";
-        retcode = HSFuncExecQuery
+        retcode = HSFuncExecTransactionalQueryWithRetry
           (query, -UERR_INTERNAL_ERROR, &rows, "CLEAR_ORPHANS",
-           NULL, NULL, TRUE/*doRetry*/ );
+           NULL, NULL);
         if (LM->LogNeeded())
           {
             convertInt64ToAscii(rows, rowCountStr);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a39e85b7/core/sql/ustat/hs_log.h
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_log.h b/core/sql/ustat/hs_log.h
index 783b96f..305ce75 100644
--- a/core/sql/ustat/hs_log.h
+++ b/core/sql/ustat/hs_log.h
@@ -78,13 +78,15 @@ public:
                        , short sqlcode
                        , const char *msg
                        , Int32 needCLIDiags
+                       , NABoolean inactivate=FALSE
                        )
     : retcode_(retcode),
       sqlcode_(sqlcode),
       string0_(msg),
       needCLIDiags_(needCLIDiags),
       string1_(NULL),
-      isFinalized_(FALSE)
+      isFinalized_(FALSE),
+      inactivated_(inactivate)
     {
       retcode_ = 0;
     }
@@ -108,6 +110,7 @@ public:
     {
       // Add error to diagnostic area if not EOF or float primary key (-1120).
       if ( !isFinalized_ &&
+           !inactivated_ &&
            retcode_ &&
            retcode_ != HS_EOF &&
            retcode_ != -HS_PKEY_FLOAT_ERROR &&
@@ -126,6 +129,16 @@ private:
   const char *string1_;    // optional string1.
   NABoolean isFinalized_;  // becomes true if error caught before object destroyed
                            //   (prevents action by dtor)
+  NABoolean inactivated_;  // Set to TRUE if we want to suppress the action
+                           // of finalize() in the destructor. We might do
+                           // this because this HSErrorCatcher was created
+                           // within the scope of another; if we don't suppress
+                           // we will report the same error twice. Another
+                           // reason for doing this is we might be in a retry
+                           // loop and don't want to report diagnostics for
+                           // failures we will retry. (In this latter case, 
+                           // there typically will be another HSErrorCatcher
+                           // above the retry loop.
 };
 
 // -----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/a39e85b7/core/sql/ustat/hs_read.cpp
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_read.cpp b/core/sql/ustat/hs_read.cpp
index 3271d95..b0f4ffd 100644
--- a/core/sql/ustat/hs_read.cpp
+++ b/core/sql/ustat/hs_read.cpp
@@ -2031,8 +2031,10 @@ Lng32 updateHistogram(const char *histogramTableName, Int32 stmtNum, short readC
   qry.append(count_str);
   qry.append(" WHERE CURRENT OF CURSOR106_MX_2300");
 
+  // Note that the UPDATE cannot be retried since the transaction will
+  // likely abort on any failure, invalidating the cursor.
   Lng32 retcode = HSFuncExecQuery(qry.data(), -UERR_INTERNAL_ERROR, NULL,
-                                  HS_QUERY_ERROR, NULL, NULL, TRUE);
+                                  HS_QUERY_ERROR, NULL, NULL);
   HSFilterWarning(retcode);
   if (LM->LogNeeded())
   {



[2/2] incubator-trafodion git commit: Merge [TRAFODION-2341] Redesign UPDATE STATISTICS retry logic

Posted by db...@apache.org.
Merge [TRAFODION-2341] Redesign UPDATE STATISTICS retry logic


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

Branch: refs/heads/master
Commit: bbbd26ec836f94069ca0cd01b0f3d7b0c2f22526
Parents: a596155 a39e85b
Author: Dave Birdsall <db...@apache.org>
Authored: Sat Nov 19 01:14:24 2016 +0000
Committer: Dave Birdsall <db...@apache.org>
Committed: Sat Nov 19 01:14:24 2016 +0000

----------------------------------------------------------------------
 core/sql/ustat/hs_cli.cpp     | 361 +++++++++++++++++++++----------------
 core/sql/ustat/hs_cli.h       |  67 ++++++-
 core/sql/ustat/hs_globals.cpp | 121 +++++--------
 core/sql/ustat/hs_log.h       |  15 +-
 core/sql/ustat/hs_read.cpp    |   4 +-
 5 files changed, 328 insertions(+), 240 deletions(-)
----------------------------------------------------------------------