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?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]