Skip to content

[GOBBLIN-2173] Avoid Adhoc flow spec addition for non leasable entity #4076

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 8 commits into from
Nov 19, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.gobblin.metrics.ServiceMetricNames;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.FlowSpecSearchObject;
import org.apache.gobblin.runtime.api.TooSoonToRerunSameFlowException;
import org.apache.gobblin.runtime.api.SpecNotFoundException;
import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse;
import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
Expand Down Expand Up @@ -256,6 +257,9 @@ public CreateKVResponse<ComplexResourceKey<FlowId, FlowStatusId>, FlowConfig> cr
responseMap = this.flowCatalog.put(flowSpec, true);
} catch (QuotaExceededException e) {
throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage());
} catch(TooSoonToRerunSameFlowException e) {
return new CreateKVResponse<>(new RestLiServiceException(HttpStatus.S_409_CONFLICT,
"FlowSpec with URI " + flowSpec.getUri() + " was launched within the lease consolidation period, no action will be taken"));
} catch (Throwable e) {
// TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings
log.warn(String.format("Failed to add flow configuration %s.%s to catalog due to", flowConfig.getId().getFlowGroup(), flowConfig.getId().getFlowName()), e);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.gobblin.runtime.api;

/**
* An {@link RuntimeException} thrown when lease cannot be acquired on provided entity.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

needs update

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated javadoc

*/
public class TooSoonToRerunSameFlowException extends RuntimeException {
private final FlowSpec flowSpec;

public TooSoonToRerunSameFlowException(String message, FlowSpec flowSpec) {
super(message);
this.flowSpec = flowSpec;
}

public FlowSpec getFlowSpec() {
return flowSpec;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.Properties;

import org.apache.commons.lang3.reflect.ConstructorUtils;
import org.apache.gobblin.runtime.api.TooSoonToRerunSameFlowException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -392,7 +393,12 @@ private Map<String, AddSpecResponse> updateOrAddSpecHelper(Spec spec, boolean tr
// If flow fails compilation, the result will have a non-empty string with the error
if (!response.getValue().getFailures().isEmpty()) {
for (Map.Entry<SpecCatalogListener, CallbackResult<AddSpecResponse>> entry : response.getValue().getFailures().entrySet()) {
throw entry.getValue().getError().getCause();
Throwable error = entry.getValue().getError();
if (error instanceof TooSoonToRerunSameFlowException) {
throw (TooSoonToRerunSameFlowException) error;
} else {
throw error.getCause();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not crazy about having to explicitly carve out a special case for this exception. couldn't we instead, when throwing it in the first place, wrap it in an extra RuntimeException that we know will be stripped off here?

(if doing that, be sure to add a comment explaining it's for the SpecCatalogListener CallbackResult handling over here.)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure updated as per suggestion

}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,16 @@ public interface DagManagementStateStore {
*/
void updateDagNode(Dag.DagNode<JobExecutionPlan> dagNode) throws IOException;

/**
* Returns true if lease can be acquired on entity provided in leaseParams.
* Check if an action exists in dagAction store by flow group, flow name, flow execution id, and job name.
* @param flowGroup flow group for the dag action
* @param flowName flow name for the dag action
* @param flowExecutionId flow execution for the dag action
* @throws IOException
*/
boolean existsCurrentlyLaunchingSimilarFlow(String flowGroup, String flowName, long flowExecutionId) throws IOException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about existsCurrentlyLaunchingExecOfSameFlow?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure updated


/**
* Returns the requested {@link org.apache.gobblin.service.modules.flowgraph.Dag.DagNode} and its {@link JobStatus}.
* Both params are returned as optional and are empty if not present in the store.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,11 @@ public LeaseAttemptStatus tryAcquireLease(DagActionStore.LeaseParams leaseParams
throw new RuntimeException(String.format("Unexpected LeaseAttemptStatus (%s) for %s", leaseAttemptStatus.getClass().getName(), leaseParams));
}

@Override
public boolean existsSimilarLeaseWithinConsolidationPeriod(DagActionStore.LeaseParams leaseParams) throws IOException {
return decoratedMultiActiveLeaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(leaseParams);
}

@Override
public boolean recordLeaseSuccess(LeaseAttemptStatus.LeaseObtainedStatus status)
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,17 @@ public interface MultiActiveLeaseArbiter {
LeaseAttemptStatus tryAcquireLease(DagActionStore.LeaseParams leaseParams, boolean adoptConsensusFlowExecutionId)
throws IOException;

/**
* This method checks if lease can be acquired on provided flow in lease params
* returns true if entry for the same flow does not exists within Lease Consolidation Period
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sense is reversed here...

maybe:

Check whether the same flowGroup+flowName is within the Lease Consolidation Period (aka. epsilon) from other, unrelated leasing activity

this is also out-of-date:

@return true if lease can be acquired on the flow passed in the lease params, false otherwise

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated javadoc

* in leaseArbiterStore, else returns false
* @param leaseParams uniquely identifies the flow, the present action upon it, the time the action
* was triggered, and if the dag action event we're checking on is a reminder event
* @return true if lease can be acquired on the flow passed in the lease params, false otherwise
*/
boolean existsSimilarLeaseWithinConsolidationPeriod(DagActionStore.LeaseParams leaseParams)
throws IOException;

/**
* This method is used to indicate the owner of the lease has successfully completed required actions while holding
* the lease of the dag action event. It marks the lease as "no longer leasing", if the eventTimeMillis and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ public class MySqlDagManagementStateStore implements DagManagementStateStore {
// todo - these two stores should merge
private DagStateStoreWithDagNodes dagStateStore;
private DagStateStoreWithDagNodes failedDagStateStore;
private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
private final JobStatusRetriever jobStatusRetriever;
private boolean dagStoresInitialized = false;
private final UserQuotaManager quotaManager;
Expand All @@ -79,13 +80,14 @@ public class MySqlDagManagementStateStore implements DagManagementStateStore {

@Inject
public MySqlDagManagementStateStore(Config config, FlowCatalog flowCatalog, UserQuotaManager userQuotaManager,
JobStatusRetriever jobStatusRetriever, DagActionStore dagActionStore) {
JobStatusRetriever jobStatusRetriever, DagActionStore dagActionStore, MultiActiveLeaseArbiter multiActiveLeaseArbiter) {
this.quotaManager = userQuotaManager;
this.config = config;
this.flowCatalog = flowCatalog;
this.jobStatusRetriever = jobStatusRetriever;
this.dagManagerMetrics.activate();
this.dagActionStore = dagActionStore;
this.multiActiveLeaseArbiter = multiActiveLeaseArbiter;
}

// It should be called after topology spec map is set
Expand Down Expand Up @@ -168,6 +170,14 @@ public synchronized void updateDagNode(Dag.DagNode<JobExecutionPlan> dagNode)
this.dagStateStore.updateDagNode(dagNode);
}

@Override
public boolean existsCurrentlyLaunchingSimilarFlow(String flowGroup, String flowName, long flowExecutionId) throws IOException {
DagActionStore.DagAction dagAction = DagActionStore.DagAction.forFlow(flowGroup, flowName,
flowExecutionId, DagActionStore.DagActionType.LAUNCH);
DagActionStore.LeaseParams leaseParams = new DagActionStore.LeaseParams(dagAction, System.currentTimeMillis());
return multiActiveLeaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(leaseParams);
}

@Override
public Optional<Dag<JobExecutionPlan>> getDag(Dag.DagId dagId) throws IOException {
return Optional.ofNullable(this.dagStateStore.getDag(dagId));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,16 @@ else if (leaseValidityStatus == 2) {
}
}

/*
Determines if a lease can be acquired for the given flow. A lease is acquirable if
no existing lease record exists in arbiter table or the record is older then epsilon time
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably no need for this comment here in the impl, but if you want one, bring it into line w/ the orig from the interface

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

makes sense, since javadoc already there for the interface, removed comment from here

@Override
public boolean existsSimilarLeaseWithinConsolidationPeriod(DagActionStore.LeaseParams leaseParams) throws IOException {
Optional<GetEventInfoResult> infoResult = getExistingEventInfo(leaseParams);
return infoResult.isPresent() ? infoResult.get().isWithinEpsilon() : false;
}

/**
* Checks leaseArbiterTable for an existing entry for this dag action and event time
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.Properties;
import java.util.concurrent.TimeUnit;

import org.apache.gobblin.runtime.api.TooSoonToRerunSameFlowException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this import belongs a few lines down w/ other apache gobblin pkgs

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -78,6 +79,7 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
protected final SpecCompiler specCompiler;
protected final TopologyCatalog topologyCatalog;
private final JobStatusRetriever jobStatusRetriever;
private final DagManagementStateStore dagManagementStateStore;

protected final MetricContext metricContext;

Expand All @@ -100,6 +102,7 @@ public Orchestrator(Config config, TopologyCatalog topologyCatalog, Optional<Log
this.topologyCatalog = topologyCatalog;
this.flowLaunchHandler = flowLaunchHandler;
this.sharedFlowMetricsSingleton = sharedFlowMetricsSingleton;
this.dagManagementStateStore = dagManagementStateStore;
this.jobStatusRetriever = jobStatusRetriever;
this.specCompiler = flowCompilationValidationHelper.getSpecCompiler();
// todo remove the need to set topology factory outside of constructor GOBBLIN-2056
Expand All @@ -125,6 +128,7 @@ public AddSpecResponse onAddSpec(Spec addedSpec) {
_log.info("Orchestrator - onAdd[Topology]Spec: " + addedSpec);
this.specCompiler.onAddSpec(addedSpec);
} else if (addedSpec instanceof FlowSpec) {
enforceSimilarAdhocFlowExistence((FlowSpec) addedSpec);
_log.info("Orchestrator - onAdd[Flow]Spec: " + addedSpec);
return this.specCompiler.onAddSpec(addedSpec);
} else {
Expand All @@ -133,6 +137,28 @@ public AddSpecResponse onAddSpec(Spec addedSpec) {
return new AddSpecResponse<>(null);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just an FYI, this also gets called during updating a flow. But since we have a condition of checking the flow is scheduled or not and we don't expect users to update an adhoc flow, we should be fine.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But since this can still be called for adhoc flows, it would be good to test what the behaviour is. No need to handle it specially, but to know what the behaviour is would be good.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the callout, will add it for the test suite


/*
enforces that a similar flow is not launching,
else throw TooSoonToRerunSameFlowException
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: {@link TooSoonToRerunSameFlowException}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure updated

*/
private void enforceSimilarAdhocFlowExistence(FlowSpec flowSpec) {
if (!flowSpec.isScheduled()) {
Config flowConfig = flowSpec.getConfig();
String flowGroup = flowConfig.getString(ConfigurationKeys.FLOW_GROUP_KEY);
String flowName = flowConfig.getString(ConfigurationKeys.FLOW_NAME_KEY);

_log.info("checking existing adhoc flow existence for " + flowGroup + "." + flowName);
try {
if (dagManagementStateStore.existsCurrentlyLaunchingSimilarFlow(flowGroup, flowName, FlowUtils.getOrCreateFlowExecutionId(flowSpec))) {
throw new TooSoonToRerunSameFlowException("Lease already occupied by another execution of this flow", flowSpec);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. we have an .info line above announcing the check, so let's follow w/ a .warn line here when the check fails. suggest: "another recent adhoc flow exec found for...."

  2. exception msg could benefit from minor improvements, yet--however it's phrased--it belongs encapsulated in the TooSoonToRerun... ctor, which should take solely a FlowSpec param

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated, Thanks for the suggestion

}
} catch (IOException exception) {
_log.error("unable to check whether similar flow exists " + flowGroup + "." + flowName);
throw new RuntimeException("unable to check whether similar flow exists " + flowGroup + "." + flowName, exception);
}
}
}

public void onDeleteSpec(URI deletedSpecURI, String deletedSpecVersion) {
onDeleteSpec(deletedSpecURI, deletedSpecVersion, new Properties());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.Map;
import java.util.Set;

import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand All @@ -47,8 +48,7 @@
import org.apache.gobblin.service.monitoring.JobStatusRetriever;
import org.apache.gobblin.util.CompletedFuture;

import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.*;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;

Expand All @@ -59,6 +59,7 @@
public class MySqlDagManagementStateStoreTest {

private ITestMetastoreDatabase testDb;
private static MultiActiveLeaseArbiter leaseArbiter;
private MySqlDagManagementStateStore dagManagementStateStore;
private static final String TEST_USER = "testUser";
public static final String TEST_PASSWORD = "testPassword";
Expand All @@ -68,6 +69,7 @@ public class MySqlDagManagementStateStoreTest {
@BeforeClass
public void setUp() throws Exception {
// Setting up mock DB
this.leaseArbiter = mock(MultiActiveLeaseArbiter.class);
this.testDb = TestMetastoreDatabaseFactory.get();
this.dagManagementStateStore = getDummyDMSS(this.testDb);
}
Expand All @@ -92,6 +94,22 @@ public static <T> boolean compareLists(List<T> list1, List<T> list2) {
return true;
}

@Test
public void testExistsCurrentlyLaunchingSimilarFlowGivesTrue() throws Exception{
Mockito.when(leaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(Mockito.any(DagActionStore.LeaseParams.class))).thenReturn(true);
String flowName = "testFlow";
String flowGroup = "testGroup";
Assert.assertTrue(dagManagementStateStore.existsCurrentlyLaunchingSimilarFlow(flowGroup, flowName, any(Long.class)));
}

@Test
public void testExistsCurrentlyLaunchingSimilarFlowGivesFalse() throws Exception{
Mockito.when(leaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(Mockito.any(DagActionStore.LeaseParams.class))).thenReturn(false);
String flowName = "testFlow";
String flowGroup = "testGroup";
Assert.assertFalse(dagManagementStateStore.existsCurrentlyLaunchingSimilarFlow(flowGroup, flowName, any(Long.class)));
}

@Test
public void testAddDag() throws Exception {
Dag<JobExecutionPlan> dag = DagTestUtils.buildDag("test", 12345L);
Expand Down Expand Up @@ -150,9 +168,11 @@ public static MySqlDagManagementStateStore getDummyDMSS(ITestMetastoreDatabase t
TopologySpec topologySpec = LaunchDagProcTest.buildNaiveTopologySpec(TEST_SPEC_EXECUTOR_URI);
URI specExecURI = new URI(TEST_SPEC_EXECUTOR_URI);
topologySpecMap.put(specExecURI, topologySpec);
MultiActiveLeaseArbiter multiActiveLeaseArbiter = Mockito.mock(MultiActiveLeaseArbiter.class);
leaseArbiter = multiActiveLeaseArbiter;
MySqlDagManagementStateStore dagManagementStateStore =
new MySqlDagManagementStateStore(config, null, null, jobStatusRetriever,
MysqlDagActionStoreTest.getTestDagActionStore(testMetastoreDatabase));
MysqlDagActionStoreTest.getTestDagActionStore(testMetastoreDatabase), multiActiveLeaseArbiter);
dagManagementStateStore.setTopologySpecMap(topologySpecMap);
return dagManagementStateStore;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
public class MysqlMultiActiveLeaseArbiterTest {
private static final long EPSILON = 10000L;
private static final long MORE_THAN_EPSILON = (long) (EPSILON * 1.1);
private static final long LESS_THAN_EPSILON = (long) (EPSILON * 0.90);
// NOTE: `sleep`ing this long SIGNIFICANTLY slows tests, but we need a large enough value that exec. variability won't cause spurious failure
private static final long LINGER = 20000L;
private static final long MORE_THAN_LINGER = (long) (LINGER * 1.1);
Expand All @@ -53,9 +54,12 @@ public class MysqlMultiActiveLeaseArbiterTest {
private static final String CONSTANTS_TABLE = "constants_store";
private static final String flowGroup = "testFlowGroup";
private static final String flowGroup2 = "testFlowGroup2";
private static final String flowGroup3 = "testFlowGroup3";
private static final String flowGroup4 = "testFlowGroup4";
private static final String flowName = "testFlowName";
private static final String jobName = "testJobName";
private static final long flowExecutionId = 12345677L;
private static final long flowExecutionId1 = 12345996L;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I never noticed before that flowExecutionId, which is customarily millis-since-epoch only has 7 digits when it should have 10. let's fix that and also define this as:

private static final long flowExecutionIdAlt = flowExecutionId + ...; // whatever you consider a reasonable (later) offset

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated the same variables to store millis

private static final long eventTimeMillis = 1710451837L;
// Dag actions with the same flow info but different flow action types are considered unique
private static final DagActionStore.DagAction launchDagAction =
Expand All @@ -70,6 +74,22 @@ public class MysqlMultiActiveLeaseArbiterTest {
new DagActionStore.DagAction(flowGroup2, flowName, flowExecutionId, jobName, DagActionStore.DagActionType.LAUNCH);
private static final DagActionStore.LeaseParams
launchLeaseParams2 = new DagActionStore.LeaseParams(launchDagAction2, false, eventTimeMillis);
private static final DagActionStore.DagAction launchDagAction3 =
new DagActionStore.DagAction(flowGroup3, flowName, flowExecutionId, jobName, DagActionStore.DagActionType.LAUNCH);
private static final DagActionStore.LeaseParams
launchLeaseParams3 = new DagActionStore.LeaseParams(launchDagAction3, false, eventTimeMillis);
private static final DagActionStore.DagAction launchDagAction4 =
new DagActionStore.DagAction(flowGroup4, flowName, flowExecutionId, jobName, DagActionStore.DagActionType.LAUNCH);
private static final DagActionStore.LeaseParams
launchLeaseParams4 = new DagActionStore.LeaseParams(launchDagAction4, false, eventTimeMillis);
private static final DagActionStore.DagAction launchDagAction3_similar =
new DagActionStore.DagAction(flowGroup3, flowName, flowExecutionId1, jobName, DagActionStore.DagActionType.LAUNCH);
private static final DagActionStore.LeaseParams
launchLeaseParams3_similar = new DagActionStore.LeaseParams(launchDagAction3_similar, false, eventTimeMillis);
private static final DagActionStore.DagAction launchDagAction4_similar =
new DagActionStore.DagAction(flowGroup4, flowName, flowExecutionId1, jobName, DagActionStore.DagActionType.LAUNCH);
private static final DagActionStore.LeaseParams
launchLeaseParams4_similar = new DagActionStore.LeaseParams(launchDagAction4_similar, false, eventTimeMillis);
private static final Timestamp dummyTimestamp = new Timestamp(99999);
private ITestMetastoreDatabase testDb;
private MysqlMultiActiveLeaseArbiter mysqlMultiActiveLeaseArbiter;
Expand Down Expand Up @@ -201,6 +221,33 @@ public void testAcquireLeaseSingleParticipant() throws Exception {
<= sixthObtainedStatus.getLeaseAcquisitionTimestamp());
}

/*
test to verify if leasable entity is unavailable before epsilon time
to account for clock drift
*/
@Test
public void testExistsSimilarLeaseWithinConsolidationPeriod() throws Exception{
LeaseAttemptStatus firstLaunchStatus =
mysqlMultiActiveLeaseArbiter.tryAcquireLease(launchLeaseParams3, true);
Assert.assertTrue(firstLaunchStatus instanceof LeaseAttemptStatus.LeaseObtainedStatus);
completeLeaseHelper(launchLeaseParams3);
Thread.sleep(LESS_THAN_EPSILON);
Assert.assertTrue(mysqlMultiActiveLeaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(launchLeaseParams3_similar));
}

/*
test to verify if leasable entity exists post epsilon time
*/
@Test
public void testDoesNotExistsSimilarLeaseWithinConsolidationPeriod() throws Exception{
LeaseAttemptStatus firstLaunchStatus =
mysqlMultiActiveLeaseArbiter.tryAcquireLease(launchLeaseParams4, true);
Assert.assertTrue(firstLaunchStatus instanceof LeaseAttemptStatus.LeaseObtainedStatus);
completeLeaseHelper(launchLeaseParams4);
Thread.sleep(MORE_THAN_EPSILON);
Assert.assertFalse(mysqlMultiActiveLeaseArbiter.existsSimilarLeaseWithinConsolidationPeriod(launchLeaseParams4_similar));
}

/*
Tests attemptLeaseIfNewRow() method to ensure a new row is inserted if no row matches the primary key in the table.
If such a row does exist, the method should disregard the resulting SQL error and return 0 rows updated, indicating
Expand Down
Loading