[ 
https://issues.apache.org/jira/browse/GOBBLIN-2096?focusedWorklogId=923980&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-923980
 ]

ASF GitHub Bot logged work on GOBBLIN-2096:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 21/Jun/24 14:22
            Start Date: 21/Jun/24 14:22
    Worklog Time Spent: 10m 
      Work Description: arjun4084346 commented on code in PR #3982:
URL: https://github.com/apache/gobblin/pull/3982#discussion_r1645217691


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -420,15 +427,25 @@ protected GetEventInfoResult 
createGetInfoResult(ResultSet resultSet) throws IOE
    * near past for it.
    * @return int corresponding to number of rows updated by INSERT statement 
to acquire lease
    */
-  protected int attemptLeaseIfNewRow(DagActionStore.DagAction dagAction) 
throws IOException {
+  protected int attemptLeaseIfNewRow(DagActionStore.DagAction dagAction, 
ExponentialBackoff exponentialBackoff) throws IOException {
     String formattedAcquireLeaseNewRowStatement =
         String.format(ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT, 
this.leaseArbiterTableName);
     return 
dbStatementExecutor.withPreparedStatement(formattedAcquireLeaseNewRowStatement,
         insertStatement -> {
           completeInsertPreparedStatement(insertStatement, dagAction);
           try {
             return insertStatement.executeUpdate();
-          } catch (SQLIntegrityConstraintViolationException e) {
+          } catch (SQLTransientException e) {
+            try {
+              if (exponentialBackoff.awaitNextRetryIfAvailable()) {
+                return attemptLeaseIfNewRow(dagAction, exponentialBackoff);
+              }
+            } catch (InterruptedException | IOException e2) {

Review Comment:
   IOException is not required, is it?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -420,15 +428,25 @@ protected GetEventInfoResult 
createGetInfoResult(ResultSet resultSet) throws IOE
    * near past for it.
    * @return int corresponding to number of rows updated by INSERT statement 
to acquire lease
    */
-  protected int attemptLeaseIfNewRow(DagActionStore.DagAction dagAction) 
throws IOException {
+  protected int attemptLeaseIfNewRow(DagActionStore.DagAction dagAction, 
ExponentialBackoff exponentialBackoff) throws IOException {
     String formattedAcquireLeaseNewRowStatement =
         String.format(ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT, 
this.leaseArbiterTableName);
     return 
dbStatementExecutor.withPreparedStatement(formattedAcquireLeaseNewRowStatement,
         insertStatement -> {
           completeInsertPreparedStatement(insertStatement, dagAction);
           try {
             return insertStatement.executeUpdate();
-          } catch (SQLIntegrityConstraintViolationException e) {
+          } catch (SQLTransientException e) {
+            try {
+              if (exponentialBackoff.awaitNextRetryIfAvailable()) {
+                return attemptLeaseIfNewRow(dagAction, exponentialBackoff);
+              }
+            } catch (InterruptedException | IOException e2) {
+              throw new IOException(e2);
+            }
+            return 0;

Review Comment:
   this line will reach when there is a continuous SQLTransientException that 
exponential backoff could not help with. in that case should we thrown the 
original exception as it is?





Issue Time Tracking
-------------------

    Worklog Id:     (was: 923980)
    Time Spent: 50m  (was: 40m)

> retry Transient SQL Exception for MALA 
> ---------------------------------------
>
>                 Key: GOBBLIN-2096
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-2096
>             Project: Apache Gobblin
>          Issue Type: Bug
>          Components: gobblin-service
>            Reporter: Urmi Mustafi
>            Assignee: Abhishek Tiwari
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> We're seeing a MySQLTransactionRollbackException: Deadlock found when trying 
> to get lock when calling MysqlMultiActiveLeaseArbiter. attemptLeaseIfNewRow() 
> in the rare case when two hosts check the lease arbiter table at the same 
> time, see there's no entry matching the current primary key and try to insert 
> it at the exact same time (both wait for lock on the same row). Normally, 
> there's slight discrepancy that results in an 
> IntegrityConstraintViolationException if one insert succeeds before the other 
> OR one finishes the write early and the other sees the entry when reading the 
> getResult of the arbiter table. This PR adds retries with exponential backoff 
> for transient SQL exceptions to the insert statement. Note, it's not needed 
> for other cases when the INSERT/UPDATE statements are conditional on a read 
> value.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to