From 29f5c2d0cc30fbc697af8d079607a5ffca2cedd1 Mon Sep 17 00:00:00 2001 From: Peiying Ye Date: Mon, 12 Jun 2023 19:25:03 -0700 Subject: [PATCH 01/20] [GOBBLIN-1840] Helix Job scheduler should not try to replace running workflow if within configured time --- .../GobblinClusterConfigurationKeys.java | 4 + .../cluster/GobblinHelixJobScheduler.java | 188 ++++++------ .../cluster/GobblinHelixJobSchedulerTest.java | 285 +++++++++++++++--- .../org.mockito.plugins.MockMaker | 1 + 4 files changed, 339 insertions(+), 139 deletions(-) create mode 100644 gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java index 31a8547aa98..dcd6c4ca6e7 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java @@ -222,4 +222,8 @@ public class GobblinClusterConfigurationKeys { public static final String CONTAINER_ID_KEY = GOBBLIN_HELIX_PREFIX + "containerId"; public static final String GOBBLIN_CLUSTER_SYSTEM_PROPERTY_PREFIX = GOBBLIN_CLUSTER_PREFIX + "sysProps"; + + // determines the update throttle + public static final String HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = "helix.job.scheduling.throttle.enabled"; + public static final boolean DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = false; } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 20f47da3e50..dd3eb7a0e15 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -18,6 +18,8 @@ package org.apache.gobblin.cluster; import java.io.IOException; +import java.time.Duration; +import java.time.Instant; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -110,17 +112,17 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private boolean startServicesCompleted; private final long helixJobStopTimeoutMillis; + private final long thresholdDiffTimeMillis; + private ConcurrentHashMap jobStartTimeMap; - public GobblinHelixJobScheduler(Config sysConfig, - HelixManager jobHelixManager, - Optional taskDriverHelixManager, - EventBus eventBus, - Path appWorkDir, List> metadataTags, - SchedulerService schedulerService, - MutableJobCatalog jobCatalog) throws Exception { + public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, + Optional taskDriverHelixManager, EventBus eventBus, Path appWorkDir, + List> metadataTags, SchedulerService schedulerService, MutableJobCatalog jobCatalog) + throws Exception { super(ConfigUtils.configToProperties(sysConfig), schedulerService); - this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); + this.commonJobProperties = + ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); this.jobHelixManager = jobHelixManager; this.taskDriverHelixManager = taskDriverHelixManager; this.eventBus = eventBus; @@ -128,48 +130,47 @@ public GobblinHelixJobScheduler(Config sysConfig, this.appWorkDir = appWorkDir; this.metadataTags = metadataTags; this.jobCatalog = jobCatalog; - this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); + this.metricContext = + Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); - int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, - ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, - ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); + int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, + ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); - this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", - this.metricContext, - metricsWindowSizeInMin); + this.launcherMetrics = + new GobblinHelixJobLauncherMetrics("launcherInScheduler", this.metricContext, metricsWindowSizeInMin); - this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, - this.metricContext, - metricsWindowSizeInMin); + this.jobSchedulerMetrics = + new GobblinHelixJobSchedulerMetrics(this.jobExecutor, this.metricContext, metricsWindowSizeInMin); - this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), - PathUtils.getRootPath(appWorkDir).toUri(), - appWorkDir.toString()); + this.jobsMapping = + new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), PathUtils.getRootPath(appWorkDir).toUri(), + appWorkDir.toString()); - this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", - this.metricContext, - metricsWindowSizeInMin, this.jobsMapping); + this.planningJobLauncherMetrics = + new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", this.metricContext, + metricsWindowSizeInMin, this.jobsMapping); - this.helixMetrics = new GobblinHelixMetrics("helixMetricsInJobScheduler", - this.metricContext, - metricsWindowSizeInMin); + this.helixMetrics = + new GobblinHelixMetrics("helixMetricsInJobScheduler", this.metricContext, metricsWindowSizeInMin); this.startServicesCompleted = false; - this.helixJobStopTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_STOP_TIMEOUT_SECONDS, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_STOP_TIMEOUT_SECONDS) * 1000; + this.helixJobStopTimeoutMillis = + ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_STOP_TIMEOUT_SECONDS, + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_STOP_TIMEOUT_SECONDS) * 1000; - this.helixWorkflowListingTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; + this.helixWorkflowListingTimeoutMillis = + ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, + GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; + this.thresholdDiffTimeMillis = 3600000; + this.jobStartTimeMap = new ConcurrentHashMap<>(); } @Override public Collection getStandardMetricsCollection() { - return ImmutableList.of(this.launcherMetrics, - this.jobSchedulerMetrics, - this.planningJobLauncherMetrics, - this.helixMetrics); + return ImmutableList.of(this.launcherMetrics, this.jobSchedulerMetrics, this.planningJobLauncherMetrics, + this.helixMetrics); } @Override @@ -187,11 +188,7 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job Thread.sleep(1000); } - scheduleJob(jobProps, - jobListener, - Maps.newHashMap(), - GobblinHelixJob.class); - + scheduleJob(jobProps, jobListener, Maps.newHashMap(), GobblinHelixJob.class); } catch (Exception e) { throw new JobException("Failed to schedule job " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e); } @@ -200,9 +197,9 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job @Override protected void startServices() throws Exception { - boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties, - GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, - String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); + boolean cleanAllDistJobs = + PropertiesUtils.getPropAsBoolean(this.properties, GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); if (cleanAllDistJobs) { for (org.apache.gobblin.configuration.State state : this.jobsMapping.getAllStates()) { @@ -215,50 +212,26 @@ protected void startServices() throws Exception { @Override public void runJob(Properties jobProps, JobListener jobListener) throws JobException { - new HelixRetriggeringJobCallable(this, - this.jobCatalog, - this.properties, - jobProps, - jobListener, - this.planningJobLauncherMetrics, - this.helixMetrics, - this.appWorkDir, - this.jobHelixManager, - this.taskDriverHelixManager, - this.jobsMapping, - this.locks, - this.metricContext).call(); + new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, + this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, + this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext).call(); } @Override - public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) - throws Exception { + public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) throws Exception { Properties combinedProps = new Properties(); combinedProps.putAll(properties); combinedProps.putAll(jobProps); - return new GobblinHelixJobLauncher(combinedProps, - this.jobHelixManager, - this.appWorkDir, - this.metadataTags, - this.jobRunningMap, - Optional.of(this.helixMetrics)); + return new GobblinHelixJobLauncher(combinedProps, this.jobHelixManager, this.appWorkDir, this.metadataTags, + this.jobRunningMap, Optional.of(this.helixMetrics)); } public Future scheduleJobImmediately(Properties jobProps, JobListener jobListener) { - HelixRetriggeringJobCallable retriggeringJob = new HelixRetriggeringJobCallable(this, - this.jobCatalog, - this.properties, - jobProps, - jobListener, - this.planningJobLauncherMetrics, - this.helixMetrics, - this.appWorkDir, - this.jobHelixManager, - this.taskDriverHelixManager, - this.jobsMapping, - this.locks, - this.metricContext); + HelixRetriggeringJobCallable retriggeringJob = + new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, + this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, + this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext); final Future future = this.jobExecutor.submit(retriggeringJob); return new Future() { @@ -318,13 +291,14 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobUri); - scheduleJob(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics)); + scheduleJob(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics)); } else { LOGGER.info("No job schedule found, so running job " + jobUri); - this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics))); + this.jobExecutor.execute( + new NonScheduledJobRunner(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics))); } + + this.jobStartTimeMap.put(jobUri, Instant.now()); } catch (JobException je) { LOGGER.error("Failed to schedule or run job " + jobUri, je); } @@ -333,16 +307,30 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { @Subscribe public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); + String jobName = updateJobArrival.getJobName(); + boolean throttleEnabled = PropertiesUtils.getPropAsBoolean(updateJobArrival.getJobConfig(), + GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY)); + + if (throttleEnabled && this.jobStartTimeMap.containsKey(jobName)) { + Instant jobStartTime = this.jobStartTimeMap.get(jobName); + Instant currTime = Instant.now(); + long durationMillis = Duration.between(jobStartTime, currTime).toMillis(); + if (durationMillis < this.thresholdDiffTimeMillis) { + return; + } + } + try { - handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), - updateJobArrival.getJobConfig())); + handleDeleteJobConfigArrival( + new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } try { - handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), - updateJobArrival.getJobConfig())); + handleNewJobConfigArrival( + new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } @@ -371,14 +359,14 @@ public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobAr } @Subscribe - public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) - throws InterruptedException { + public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) throws InterruptedException { String jobUri = cancelJobArrival.getJoburi(); LOGGER.info("Received cancel for job configuration of job " + jobUri); Optional distributedJobMode; Optional planningJob = Optional.empty(); Optional actualJob = Optional.empty(); - boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, + boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, + GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); this.jobSchedulerMetrics.numCancellationStart.incrementAndGet(); @@ -397,7 +385,8 @@ public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobAr if (planningJob.isPresent()) { LOGGER.info("Cancelling planning job helix workflow: {}", planningJob.get()); - HelixUtils.cancelWorkflow(planningJob.get(), this.taskDriverHelixManager.get(), this.helixJobStopTimeoutMillis, cancelByDelete); + HelixUtils.cancelWorkflow(planningJob.get(), this.taskDriverHelixManager.get(), this.helixJobStopTimeoutMillis, + cancelByDelete); } if (actualJob.isPresent()) { @@ -415,12 +404,15 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t LOGGER.info("Cancelling workflow: {}", deleteJobArrival.getJobName()); //Workaround for preventing indefinite hangs observed in TaskDriver.getWorkflows() call. - Callable> workflowsCallable = () -> HelixUtils.getWorkflowIdsFromJobNames(this.jobHelixManager, - Collections.singletonList(deleteJobArrival.getJobName())); + Callable> workflowsCallable = + () -> HelixUtils.getWorkflowIdsFromJobNames(this.jobHelixManager, + Collections.singletonList(deleteJobArrival.getJobName())); Retryer> retryer = RetryerBuilder.>newBuilder() .retryIfException() .withStopStrategy(StopStrategies.stopAfterAttempt(5)) - .withAttemptTimeLimiter(AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)).build(); + .withAttemptTimeLimiter( + AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)) + .build(); Map jobNameToWorkflowIdMap; try { jobNameToWorkflowIdMap = retryer.call(workflowsCallable); @@ -431,8 +423,9 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t if (jobNameToWorkflowIdMap.containsKey(deleteJobArrival.getJobName())) { String workflowId = jobNameToWorkflowIdMap.get(deleteJobArrival.getJobName()); - boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(jobConfig, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, - String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); + boolean cancelByDelete = + PropertiesUtils.getPropAsBoolean(jobConfig, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); HelixUtils.cancelWorkflow(workflowId, this.jobHelixManager, helixJobStopTimeoutMillis, cancelByDelete); LOGGER.info("Cancelled workflow: {}", deleteJobArrival.getJobName()); //Wait until the cancelled job is complete. @@ -442,7 +435,6 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t } } } - /** * This class is responsible for running non-scheduled jobs. */ @@ -451,8 +443,7 @@ class NonScheduledJobRunner implements Runnable { private final GobblinHelixJobLauncherListener jobListener; private final Long creationTimeInMillis; - public NonScheduledJobRunner(Properties jobProps, - GobblinHelixJobLauncherListener jobListener) { + public NonScheduledJobRunner(Properties jobProps, GobblinHelixJobLauncherListener jobListener) { this.jobProps = jobProps; this.jobListener = jobListener; @@ -463,7 +454,8 @@ public NonScheduledJobRunner(Properties jobProps, public void run() { try { GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBeforeJobLaunching(this.jobProps); - GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis()); + GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching( + this.creationTimeInMillis, System.currentTimeMillis()); GobblinHelixJobScheduler.this.runJob(this.jobProps, this.jobListener); } catch (JobException je) { LOGGER.error("Failed to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 498e2530aec..77e4b1ecef7 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; import java.net.URL; +import java.nio.file.Files; +import java.time.Instant; import java.util.Collections; import java.util.Map; import java.util.Properties; @@ -32,6 +34,7 @@ import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.assertj.core.util.Lists; +import org.mockito.MockedStatic; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -52,6 +55,8 @@ import org.apache.gobblin.runtime.job_catalog.NonObservingFSJobCatalog; import org.apache.gobblin.scheduler.SchedulerService; +import static org.mockito.Mockito.*; + /** * Unit tests for {@link org.apache.gobblin.cluster.GobblinHelixJobScheduler}. @@ -70,9 +75,14 @@ public class GobblinHelixJobSchedulerTest { private GobblinTaskRunner gobblinTaskRunner; private Thread thread; - private final String workflowIdSuffix1 = "_1504201348471"; private final String workflowIdSuffix2 = "_1504201348472"; + private final String workflowIdSuffix3 = "_1504201348473"; + private final String workflowIdSuffix4 = "_1504201348474"; + + private Instant beginTime = Instant.ofEpochMilli(0); + private Instant shortPeriod = Instant.ofEpochMilli(1); + private Instant longPeriod = Instant.ofEpochMilli(3600001); @BeforeClass public void setUp() @@ -129,58 +139,251 @@ public void setUp() this.thread.start(); } - @Test - public void testNewJobAndUpdate() + public String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent) throws Exception { - Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, - ConfigValueFactory.fromAnyRef("/tmp/" + GobblinHelixJobScheduler.class.getSimpleName())); - SchedulerService schedulerService = new SchedulerService(new Properties()); - NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); - jobCatalog.startAsync(); - GobblinHelixJobScheduler jobScheduler = - new GobblinHelixJobScheduler(ConfigFactory.empty(), this.helixManager, java.util.Optional.empty(), - new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); - - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties(this.baseConfig, "1", workflowIdSuffix1); - properties1.setProperty(GobblinClusterConfigurationKeys.CANCEL_RUNNING_JOB_ON_DELETE, "true"); - - NewJobConfigArrivalEvent newJobConfigArrivalEvent = - new NewJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - properties1.setProperty(ConfigurationKeys.JOB_ID_KEY, - "job_" + properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY) + workflowIdSuffix2); - Map workflowIdMap; - this.helixManager.connect(); - - String workFlowId = null; long endTime = System.currentTimeMillis() + 30000; + Map workflowIdMap; while (System.currentTimeMillis() < endTime) { workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(this.helixManager, Collections.singletonList(newJobConfigArrivalEvent.getJobName())); if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - workFlowId = workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); - break; + return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); } Thread.sleep(100); } + return null; + } + + private GobblinHelixJobScheduler createJobScheduler() throws Exception { + java.nio.file.Path p = Files.createTempDirectory(GobblinHelixJobScheduler.class.getSimpleName()); + Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, + ConfigValueFactory.fromAnyRef(p.toString())); + SchedulerService schedulerService = new SchedulerService(new Properties()); + NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); + jobCatalog.startAsync(); + return new GobblinHelixJobScheduler(ConfigFactory.empty(), this.helixManager, java.util.Optional.empty(), + new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); + } + + private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properties, String suffix) { + properties.setProperty(GobblinClusterConfigurationKeys.CANCEL_RUNNING_JOB_ON_DELETE, "true"); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = + new NewJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties); + properties.setProperty(ConfigurationKeys.JOB_ID_KEY, + "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + suffix); + return newJobConfigArrivalEvent; + } + + private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent) throws Exception { + helixManager.connect(); + String workFlowId = getWorkflowID(newJobConfigArrivalEvent); Assert.assertNotNull(workFlowId); - Assert.assertTrue(workFlowId.endsWith(workflowIdSuffix1)); + Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); + } - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - this.helixManager.connect(); - endTime = System.currentTimeMillis() + 30000; - while (System.currentTimeMillis() < endTime) { - workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(this.helixManager, - Collections.singletonList(newJobConfigArrivalEvent.getJobName())); - if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - workFlowId = workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); - break; - } - Thread.sleep(100); + + @Test + public void testNewJobAndUpdate() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "NewJobAndUpdate", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); + + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + } + } + + @Test + public void testUpdateSameWorkflowShortPeriodThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateSameWorkflowShortPeriodThrottle", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); + + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + } + } + + @Test + public void testUpdateSameWorkflowLongPeriodNoThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateSameWorkflowLongPeriodNoThrottle", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); + + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + } + } + + @Test + public void testUpdateSameWorkflowShortPeriodNoThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateSameWorkflowShortPeriodNoThrottle", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); + + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + } + } + + public void testUpdateDiffWorkflowShortPeriodThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle1", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle2", workflowIdSuffix3); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + } + } + + @Test + public void testUpdateDiffWorkflowShortPeriodNoThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle1", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle2", workflowIdSuffix3); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + } + } + + @Test + public void testUpdateDiffWorkflowLongPeriodThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle1", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle2", workflowIdSuffix3); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + } + } + + @Test + public void testUpdateDiffWorkflowLongPeriodNoThrottle() + throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + + final Properties properties1 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle1", workflowIdSuffix1); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle2", workflowIdSuffix3); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); } - Assert.assertTrue(workFlowId.endsWith(workflowIdSuffix2)); } @AfterClass diff --git a/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 00000000000..ca6ee9cea8e --- /dev/null +++ b/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1 @@ +mock-maker-inline \ No newline at end of file From 45c87ff1ccb46da3dd1139fb2504999d38e0c749 Mon Sep 17 00:00:00 2001 From: Peiying Ye Date: Tue, 13 Jun 2023 15:43:47 -0700 Subject: [PATCH 02/20] [GOBBLIN-1840] Remove unnecessary files --- .../apache/gobblin/cluster/HelixUtils.java | 6 +- .../cluster/GobblinHelixJobSchedulerTest.java | 104 +++++++++++------- 2 files changed, 72 insertions(+), 38 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixUtils.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixUtils.java index d6200296717..688c8c7babb 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixUtils.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixUtils.java @@ -417,7 +417,11 @@ public static Map getWorkflowIdsFromJobNames(TaskDriver taskDriv } Set helixJobs = workflowConfig.getJobDag().getAllNodes(); for (String helixJob : helixJobs) { - Iterator taskConfigIterator = taskDriver.getJobConfig(helixJob).getTaskConfigMap().values().iterator(); + JobConfig jobConfig = taskDriver.getJobConfig(helixJob); + if (jobConfig == null) { + throw new GobblinHelixUnexpectedStateException("Received null jobConfig from Helix. We should not see any null configs when reading all helixJobs. helixJob=%s", helixJob); + } + Iterator taskConfigIterator = jobConfig.getTaskConfigMap().values().iterator(); if (taskConfigIterator.hasNext()) { TaskConfig taskConfig = taskConfigIterator.next(); String jobName = taskConfig.getConfigMap().get(ConfigurationKeys.JOB_NAME_KEY); diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 77e4b1ecef7..61a37939a41 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -66,7 +66,6 @@ public class GobblinHelixJobSchedulerTest { public final static Logger LOG = LoggerFactory.getLogger(GobblinHelixJobSchedulerTest.class); - private HelixManager helixManager; private FileSystem localFs; private Path appWorkDir; private final Closer closer = Closer.create(); @@ -78,12 +77,14 @@ public class GobblinHelixJobSchedulerTest { private final String workflowIdSuffix1 = "_1504201348471"; private final String workflowIdSuffix2 = "_1504201348472"; private final String workflowIdSuffix3 = "_1504201348473"; - private final String workflowIdSuffix4 = "_1504201348474"; private Instant beginTime = Instant.ofEpochMilli(0); private Instant shortPeriod = Instant.ofEpochMilli(1); private Instant longPeriod = Instant.ofEpochMilli(3600001); + private String zkConnectingString; + private String helixClusterName; + @BeforeClass public void setUp() throws Exception { @@ -106,16 +107,16 @@ public void setUp() ConfigValueFactory.fromAnyRef(sourceJsonFile.getAbsolutePath())) .withValue(ConfigurationKeys.JOB_STATE_IN_STATE_STORE, ConfigValueFactory.fromAnyRef("true")).resolve(); - String zkConnectingString = baseConfig.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY); - String helixClusterName = baseConfig.getString(GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY); + zkConnectingString = baseConfig.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY); + helixClusterName = baseConfig.getString(GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY); HelixUtils.createGobblinHelixCluster(zkConnectingString, helixClusterName); - this.helixManager = HelixManagerFactory + HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); this.closer.register(() -> helixManager.disconnect()); - this.helixManager.connect(); + helixManager.connect(); this.localFs = FileSystem.getLocal(new Configuration()); @@ -139,13 +140,17 @@ public void setUp() this.thread.start(); } - public String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent) + public String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { long endTime = System.currentTimeMillis() + 30000; Map workflowIdMap; while (System.currentTimeMillis() < endTime) { - workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(this.helixManager, - Collections.singletonList(newJobConfigArrivalEvent.getJobName())); + try{ + workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, + Collections.singletonList(newJobConfigArrivalEvent.getJobName())); + } catch(GobblinHelixUnexpectedStateException e){ + continue; + } if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); } @@ -154,14 +159,14 @@ public String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent) return null; } - private GobblinHelixJobScheduler createJobScheduler() throws Exception { + private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager) throws Exception { java.nio.file.Path p = Files.createTempDirectory(GobblinHelixJobScheduler.class.getSimpleName()); Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, ConfigValueFactory.fromAnyRef(p.toString())); SchedulerService schedulerService = new SchedulerService(new Properties()); NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); jobCatalog.startAsync(); - return new GobblinHelixJobScheduler(ConfigFactory.empty(), this.helixManager, java.util.Optional.empty(), + return new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); } @@ -174,9 +179,9 @@ private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properti return newJobConfigArrivalEvent; } - private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent) throws Exception { + private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { helixManager.connect(); - String workFlowId = getWorkflowID(newJobConfigArrivalEvent); + String workFlowId = getWorkflowID(newJobConfigArrivalEvent, helixManager); Assert.assertNotNull(workFlowId); Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); } @@ -187,8 +192,11 @@ public void testNewJobAndUpdate() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( @@ -196,13 +204,13 @@ public void testNewJobAndUpdate() NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); } } @@ -211,21 +219,25 @@ public void testUpdateSameWorkflowShortPeriodThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); + + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowShortPeriodThrottle", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); } } @@ -234,21 +246,24 @@ public void testUpdateSameWorkflowLongPeriodNoThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowLongPeriodNoThrottle", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); } } @@ -257,21 +272,24 @@ public void testUpdateSameWorkflowShortPeriodNoThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowShortPeriodNoThrottle", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); } } @@ -279,15 +297,18 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle1", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); final Properties properties2 = GobblinHelixJobLauncherTest.generateJobProperties( @@ -298,7 +319,7 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -307,15 +328,18 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle1", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); final Properties properties2 = GobblinHelixJobLauncherTest.generateJobProperties( @@ -326,7 +350,7 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -335,15 +359,18 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle1", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); final Properties properties2 = GobblinHelixJobLauncherTest.generateJobProperties( @@ -354,7 +381,7 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -363,15 +390,18 @@ public void testUpdateDiffWorkflowLongPeriodNoThrottle() throws Exception { try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { mocked.when(Instant::now).thenReturn(beginTime, longPeriod); + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle1", workflowIdSuffix1); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent); + connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); final Properties properties2 = GobblinHelixJobLauncherTest.generateJobProperties( @@ -382,7 +412,7 @@ public void testUpdateDiffWorkflowLongPeriodNoThrottle() jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2); + connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } From df182ff56ba78c491c9ecc414b1d662f303a252f Mon Sep 17 00:00:00 2001 From: Peiying Ye Date: Tue, 13 Jun 2023 16:35:41 -0700 Subject: [PATCH 03/20] [GOBBLIN-1840] Add config for throttleTimeoutDuration --- .../cluster/GobblinClusterConfigurationKeys.java | 5 ++++- .../gobblin/cluster/GobblinHelixJobScheduler.java | 13 ++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java index dcd6c4ca6e7..fec6d27360d 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java @@ -161,6 +161,9 @@ public class GobblinClusterConfigurationKeys { public static final String HELIX_WORKFLOW_SUBMISSION_TIMEOUT_SECONDS = GOBBLIN_CLUSTER_PREFIX + "workflowSubmissionTimeoutSeconds"; public static final long DEFAULT_HELIX_WORKFLOW_SUBMISSION_TIMEOUT_SECONDS = 300; + public static final String THROTTLE_TIMEOUT_DURATION = GOBBLIN_CLUSTER_PREFIX + "throttleTimeoutDuration"; + public static final long DEFAULT_THROTTLE_TIMEOUT_DURATION = 3600; + public static final String HELIX_WORKFLOW_DELETE_TIMEOUT_SECONDS = GOBBLIN_CLUSTER_PREFIX + "workflowDeleteTimeoutSeconds"; public static final long DEFAULT_HELIX_WORKFLOW_DELETE_TIMEOUT_SECONDS = 300; @@ -223,7 +226,7 @@ public class GobblinClusterConfigurationKeys { public static final String GOBBLIN_CLUSTER_SYSTEM_PROPERTY_PREFIX = GOBBLIN_CLUSTER_PREFIX + "sysProps"; - // determines the update throttle public static final String HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = "helix.job.scheduling.throttle.enabled"; public static final boolean DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = false; + } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index dd3eb7a0e15..a91b98d7f48 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.time.Duration; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -112,7 +113,7 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private boolean startServicesCompleted; private final long helixJobStopTimeoutMillis; - private final long thresholdDiffTimeMillis; + private final Duration throttleTimeoutDuration; private ConcurrentHashMap jobStartTimeMap; public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, @@ -163,7 +164,9 @@ public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; - this.thresholdDiffTimeMillis = 3600000; + this.throttleTimeoutDuration = Duration.of( + ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.THROTTLE_TIMEOUT_DURATION, + GobblinClusterConfigurationKeys.DEFAULT_THROTTLE_TIMEOUT_DURATION), ChronoUnit.SECONDS); this.jobStartTimeMap = new ConcurrentHashMap<>(); } @@ -314,9 +317,9 @@ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobAr if (throttleEnabled && this.jobStartTimeMap.containsKey(jobName)) { Instant jobStartTime = this.jobStartTimeMap.get(jobName); - Instant currTime = Instant.now(); - long durationMillis = Duration.between(jobStartTime, currTime).toMillis(); - if (durationMillis < this.thresholdDiffTimeMillis) { + Duration workflowDuration = Duration.between(jobStartTime, Instant.now()); + Duration difference = workflowDuration.minus(throttleTimeoutDuration); + if (difference.isNegative()) { return; } } From 3c552b71a7ebc105fd5dcd35cd9b1d05823d383d Mon Sep 17 00:00:00 2001 From: Peiying Ye Date: Tue, 13 Jun 2023 17:42:50 -0700 Subject: [PATCH 04/20] [GOBBLIN-1840] Clean up format and coding standard --- .../GobblinClusterConfigurationKeys.java | 6 +- .../cluster/GobblinHelixJobScheduler.java | 104 ++++++---------- .../cluster/GobblinHelixJobSchedulerTest.java | 113 ++++++++---------- 3 files changed, 88 insertions(+), 135 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java index fec6d27360d..7e6953df564 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java @@ -161,9 +161,6 @@ public class GobblinClusterConfigurationKeys { public static final String HELIX_WORKFLOW_SUBMISSION_TIMEOUT_SECONDS = GOBBLIN_CLUSTER_PREFIX + "workflowSubmissionTimeoutSeconds"; public static final long DEFAULT_HELIX_WORKFLOW_SUBMISSION_TIMEOUT_SECONDS = 300; - public static final String THROTTLE_TIMEOUT_DURATION = GOBBLIN_CLUSTER_PREFIX + "throttleTimeoutDuration"; - public static final long DEFAULT_THROTTLE_TIMEOUT_DURATION = 3600; - public static final String HELIX_WORKFLOW_DELETE_TIMEOUT_SECONDS = GOBBLIN_CLUSTER_PREFIX + "workflowDeleteTimeoutSeconds"; public static final long DEFAULT_HELIX_WORKFLOW_DELETE_TIMEOUT_SECONDS = 300; @@ -229,4 +226,7 @@ public class GobblinClusterConfigurationKeys { public static final String HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = "helix.job.scheduling.throttle.enabled"; public static final boolean DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = false; + public static final String HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY = "helix.job.scheduling.throttle.timeout.seconds"; + public static final long DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY = 3600; + } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index a91b98d7f48..f4a005d4519 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -116,14 +116,16 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private final Duration throttleTimeoutDuration; private ConcurrentHashMap jobStartTimeMap; - public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, - Optional taskDriverHelixManager, EventBus eventBus, Path appWorkDir, - List> metadataTags, SchedulerService schedulerService, MutableJobCatalog jobCatalog) - throws Exception { + public GobblinHelixJobScheduler(Config sysConfig, + HelixManager jobHelixManager, + Optional taskDriverHelixManager, + EventBus eventBus, + Path appWorkDir, List> metadataTags, + SchedulerService schedulerService, + MutableJobCatalog jobCatalog) throws Exception { super(ConfigUtils.configToProperties(sysConfig), schedulerService); - this.commonJobProperties = - ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); + this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); this.jobHelixManager = jobHelixManager; this.taskDriverHelixManager = taskDriverHelixManager; this.eventBus = eventBus; @@ -131,49 +133,38 @@ public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, this.appWorkDir = appWorkDir; this.metadataTags = metadataTags; this.jobCatalog = jobCatalog; - this.metricContext = - Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); + this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); - int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, - ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); + int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); - this.launcherMetrics = - new GobblinHelixJobLauncherMetrics("launcherInScheduler", this.metricContext, metricsWindowSizeInMin); + this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", this.metricContext, metricsWindowSizeInMin); - this.jobSchedulerMetrics = - new GobblinHelixJobSchedulerMetrics(this.jobExecutor, this.metricContext, metricsWindowSizeInMin); + this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, this.metricContext, metricsWindowSizeInMin); - this.jobsMapping = - new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), PathUtils.getRootPath(appWorkDir).toUri(), + this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), PathUtils.getRootPath(appWorkDir).toUri(), appWorkDir.toString()); - this.planningJobLauncherMetrics = - new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", this.metricContext, + this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", this.metricContext, metricsWindowSizeInMin, this.jobsMapping); - this.helixMetrics = - new GobblinHelixMetrics("helixMetricsInJobScheduler", this.metricContext, metricsWindowSizeInMin); + this.helixMetrics = new GobblinHelixMetrics("helixMetricsInJobScheduler", this.metricContext, metricsWindowSizeInMin); this.startServicesCompleted = false; - this.helixJobStopTimeoutMillis = - ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_STOP_TIMEOUT_SECONDS, + this.helixJobStopTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_STOP_TIMEOUT_SECONDS, GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_STOP_TIMEOUT_SECONDS) * 1000; - this.helixWorkflowListingTimeoutMillis = - ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, + this.helixWorkflowListingTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; - this.throttleTimeoutDuration = Duration.of( - ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.THROTTLE_TIMEOUT_DURATION, - GobblinClusterConfigurationKeys.DEFAULT_THROTTLE_TIMEOUT_DURATION), ChronoUnit.SECONDS); + this.throttleTimeoutDuration = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); this.jobStartTimeMap = new ConcurrentHashMap<>(); } @Override public Collection getStandardMetricsCollection() { - return ImmutableList.of(this.launcherMetrics, this.jobSchedulerMetrics, this.planningJobLauncherMetrics, - this.helixMetrics); + return ImmutableList.of(this.launcherMetrics, this.jobSchedulerMetrics, this.planningJobLauncherMetrics, this.helixMetrics); } @Override @@ -200,8 +191,7 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job @Override protected void startServices() throws Exception { - boolean cleanAllDistJobs = - PropertiesUtils.getPropAsBoolean(this.properties, GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, + boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties, GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); if (cleanAllDistJobs) { @@ -215,26 +205,21 @@ protected void startServices() throws Exception { @Override public void runJob(Properties jobProps, JobListener jobListener) throws JobException { - new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, - this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, - this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext).call(); + new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext).call(); } - @Override - public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) throws Exception { + @Override public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) throws Exception { Properties combinedProps = new Properties(); combinedProps.putAll(properties); combinedProps.putAll(jobProps); - return new GobblinHelixJobLauncher(combinedProps, this.jobHelixManager, this.appWorkDir, this.metadataTags, - this.jobRunningMap, Optional.of(this.helixMetrics)); + return new GobblinHelixJobLauncher(combinedProps, this.jobHelixManager, this.appWorkDir, this.metadataTags, this.jobRunningMap, Optional.of(this.helixMetrics)); } public Future scheduleJobImmediately(Properties jobProps, JobListener jobListener) { - HelixRetriggeringJobCallable retriggeringJob = - new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, - this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, - this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext); + HelixRetriggeringJobCallable retriggeringJob = new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, + this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, + this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext); final Future future = this.jobExecutor.submit(retriggeringJob); return new Future() { @@ -297,8 +282,7 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { scheduleJob(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics)); } else { LOGGER.info("No job schedule found, so running job " + jobUri); - this.jobExecutor.execute( - new NonScheduledJobRunner(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics))); + this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics))); } this.jobStartTimeMap.put(jobUri, Instant.now()); @@ -325,15 +309,13 @@ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobAr } try { - handleDeleteJobConfigArrival( - new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); + handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } try { - handleNewJobConfigArrival( - new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); + handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } @@ -361,15 +343,13 @@ public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobAr } } - @Subscribe - public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) throws InterruptedException { + @Subscribe public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) throws InterruptedException { String jobUri = cancelJobArrival.getJoburi(); LOGGER.info("Received cancel for job configuration of job " + jobUri); Optional distributedJobMode; Optional planningJob = Optional.empty(); Optional actualJob = Optional.empty(); - boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, - GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, + boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); this.jobSchedulerMetrics.numCancellationStart.incrementAndGet(); @@ -388,8 +368,7 @@ public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobAr if (planningJob.isPresent()) { LOGGER.info("Cancelling planning job helix workflow: {}", planningJob.get()); - HelixUtils.cancelWorkflow(planningJob.get(), this.taskDriverHelixManager.get(), this.helixJobStopTimeoutMillis, - cancelByDelete); + HelixUtils.cancelWorkflow(planningJob.get(), this.taskDriverHelixManager.get(), this.helixJobStopTimeoutMillis, cancelByDelete); } if (actualJob.isPresent()) { @@ -407,15 +386,11 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t LOGGER.info("Cancelling workflow: {}", deleteJobArrival.getJobName()); //Workaround for preventing indefinite hangs observed in TaskDriver.getWorkflows() call. - Callable> workflowsCallable = - () -> HelixUtils.getWorkflowIdsFromJobNames(this.jobHelixManager, + Callable> workflowsCallable = () -> HelixUtils.getWorkflowIdsFromJobNames(this.jobHelixManager, Collections.singletonList(deleteJobArrival.getJobName())); Retryer> retryer = RetryerBuilder.>newBuilder() .retryIfException() - .withStopStrategy(StopStrategies.stopAfterAttempt(5)) - .withAttemptTimeLimiter( - AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)) - .build(); + .withStopStrategy(StopStrategies.stopAfterAttempt(5)).withAttemptTimeLimiter(AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)).build(); Map jobNameToWorkflowIdMap; try { jobNameToWorkflowIdMap = retryer.call(workflowsCallable); @@ -426,8 +401,7 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t if (jobNameToWorkflowIdMap.containsKey(deleteJobArrival.getJobName())) { String workflowId = jobNameToWorkflowIdMap.get(deleteJobArrival.getJobName()); - boolean cancelByDelete = - PropertiesUtils.getPropAsBoolean(jobConfig, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, + boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(jobConfig, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); HelixUtils.cancelWorkflow(workflowId, this.jobHelixManager, helixJobStopTimeoutMillis, cancelByDelete); LOGGER.info("Cancelled workflow: {}", deleteJobArrival.getJobName()); @@ -446,8 +420,7 @@ class NonScheduledJobRunner implements Runnable { private final GobblinHelixJobLauncherListener jobListener; private final Long creationTimeInMillis; - public NonScheduledJobRunner(Properties jobProps, GobblinHelixJobLauncherListener jobListener) { - + public NonScheduledJobRunner(Properties jobProps,GobblinHelixJobLauncherListener jobListener) { this.jobProps = jobProps; this.jobListener = jobListener; this.creationTimeInMillis = System.currentTimeMillis(); @@ -457,8 +430,7 @@ public NonScheduledJobRunner(Properties jobProps, GobblinHelixJobLauncherListene public void run() { try { GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBeforeJobLaunching(this.jobProps); - GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching( - this.creationTimeInMillis, System.currentTimeMillis()); + GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis()); GobblinHelixJobScheduler.this.runJob(this.jobProps, this.jobListener); } catch (JobException je) { LOGGER.error("Failed to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 61a37939a41..3578d02d7a6 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -140,53 +140,6 @@ public void setUp() this.thread.start(); } - public String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) - throws Exception { - long endTime = System.currentTimeMillis() + 30000; - Map workflowIdMap; - while (System.currentTimeMillis() < endTime) { - try{ - workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, - Collections.singletonList(newJobConfigArrivalEvent.getJobName())); - } catch(GobblinHelixUnexpectedStateException e){ - continue; - } - if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); - } - Thread.sleep(100); - } - return null; - } - - private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager) throws Exception { - java.nio.file.Path p = Files.createTempDirectory(GobblinHelixJobScheduler.class.getSimpleName()); - Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, - ConfigValueFactory.fromAnyRef(p.toString())); - SchedulerService schedulerService = new SchedulerService(new Properties()); - NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); - jobCatalog.startAsync(); - return new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), - new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); - } - - private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properties, String suffix) { - properties.setProperty(GobblinClusterConfigurationKeys.CANCEL_RUNNING_JOB_ON_DELETE, "true"); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = - new NewJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties); - properties.setProperty(ConfigurationKeys.JOB_ID_KEY, - "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + suffix); - return newJobConfigArrivalEvent; - } - - private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { - helixManager.connect(); - String workFlowId = getWorkflowID(newJobConfigArrivalEvent, helixManager); - Assert.assertNotNull(workFlowId); - Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); - } - - @Test public void testNewJobAndUpdate() throws Exception { @@ -225,7 +178,6 @@ public void testUpdateSameWorkflowShortPeriodThrottle() GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowShortPeriodThrottle", workflowIdSuffix1); @@ -249,9 +201,7 @@ public void testUpdateSameWorkflowLongPeriodNoThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowLongPeriodNoThrottle", workflowIdSuffix1); @@ -262,7 +212,6 @@ public void testUpdateSameWorkflowLongPeriodNoThrottle() properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); } } @@ -275,9 +224,7 @@ public void testUpdateSameWorkflowShortPeriodNoThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateSameWorkflowShortPeriodNoThrottle", workflowIdSuffix1); @@ -288,7 +235,6 @@ public void testUpdateSameWorkflowShortPeriodNoThrottle() properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); } } @@ -300,9 +246,7 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle1", workflowIdSuffix1); @@ -318,7 +262,6 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -331,9 +274,7 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle1", workflowIdSuffix1); @@ -349,7 +290,6 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -362,9 +302,7 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle1", workflowIdSuffix1); @@ -380,7 +318,6 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } @@ -393,9 +330,7 @@ public void testUpdateDiffWorkflowLongPeriodNoThrottle() HelixManager helixManager = HelixManagerFactory .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = GobblinHelixJobLauncherTest.generateJobProperties( this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle1", workflowIdSuffix1); @@ -411,11 +346,57 @@ public void testUpdateDiffWorkflowLongPeriodNoThrottle() properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); } } + private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager) throws Exception { + java.nio.file.Path p = Files.createTempDirectory(GobblinHelixJobScheduler.class.getSimpleName()); + Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, + ConfigValueFactory.fromAnyRef(p.toString())); + SchedulerService schedulerService = new SchedulerService(new Properties()); + NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); + jobCatalog.startAsync(); + return new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), + new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); + } + + private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properties, String suffix) { + properties.setProperty(GobblinClusterConfigurationKeys.CANCEL_RUNNING_JOB_ON_DELETE, "true"); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = + new NewJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties); + properties.setProperty(ConfigurationKeys.JOB_ID_KEY, + "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + suffix); + return newJobConfigArrivalEvent; + } + + private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { + helixManager.connect(); + String workFlowId = getWorkflowID(newJobConfigArrivalEvent, helixManager); + Assert.assertNotNull(workFlowId); + Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); + } + + private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) + throws Exception { + long endTime = System.currentTimeMillis() + 30000; + Map workflowIdMap; + while (System.currentTimeMillis() < endTime) { + try{ + workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, + Collections.singletonList(newJobConfigArrivalEvent.getJobName())); + } catch(GobblinHelixUnexpectedStateException e){ + continue; + } + if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { + return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); + } + Thread.sleep(100); + } + return null; + } + + @AfterClass public void tearDown() throws IOException { From 88b9a023f920ed2d58ec9c18dda1f3bb9a510d6c Mon Sep 17 00:00:00 2001 From: Peiying Ye Date: Tue, 13 Jun 2023 22:27:10 -0700 Subject: [PATCH 05/20] [GOBBLIN-1840] Clean up format layout --- .../cluster/GobblinHelixJobScheduler.java | 121 +++++++++++++----- 1 file changed, 90 insertions(+), 31 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index f4a005d4519..60d122c4ca2 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -135,36 +135,51 @@ public GobblinHelixJobScheduler(Config sysConfig, this.jobCatalog = jobCatalog; this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); - int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); + int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, + ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, + ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); - this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", this.metricContext, metricsWindowSizeInMin); + this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", + this.metricContext, + metricsWindowSizeInMin); - this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, this.metricContext, metricsWindowSizeInMin); + this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, + this.metricContext, + metricsWindowSizeInMin); - this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), PathUtils.getRootPath(appWorkDir).toUri(), - appWorkDir.toString()); + this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), + PathUtils.getRootPath(appWorkDir).toUri(), + appWorkDir.toString()); - this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", this.metricContext, - metricsWindowSizeInMin, this.jobsMapping); + this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", + this.metricContext, + metricsWindowSizeInMin, this.jobsMapping); - this.helixMetrics = new GobblinHelixMetrics("helixMetricsInJobScheduler", this.metricContext, metricsWindowSizeInMin); + this.helixMetrics = new GobblinHelixMetrics("helixMetricsInJobScheduler", + this.metricContext, + metricsWindowSizeInMin); this.startServicesCompleted = false; this.helixJobStopTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_STOP_TIMEOUT_SECONDS, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_STOP_TIMEOUT_SECONDS) * 1000; + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_STOP_TIMEOUT_SECONDS) * 1000; this.helixWorkflowListingTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; + GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; this.throttleTimeoutDuration = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + this.jobStartTimeMap = new ConcurrentHashMap<>(); + } @Override public Collection getStandardMetricsCollection() { - return ImmutableList.of(this.launcherMetrics, this.jobSchedulerMetrics, this.planningJobLauncherMetrics, this.helixMetrics); + return ImmutableList.of(this.launcherMetrics, + this.jobSchedulerMetrics, + this.planningJobLauncherMetrics, + this.helixMetrics); } @Override @@ -182,7 +197,11 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job Thread.sleep(1000); } - scheduleJob(jobProps, jobListener, Maps.newHashMap(), GobblinHelixJob.class); + scheduleJob(jobProps, + jobListener, + Maps.newHashMap(), + GobblinHelixJob.class); + } catch (Exception e) { throw new JobException("Failed to schedule job " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e); } @@ -191,8 +210,9 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job @Override protected void startServices() throws Exception { - boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties, GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, - String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); + boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties, + GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS, + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS)); if (cleanAllDistJobs) { for (org.apache.gobblin.configuration.State state : this.jobsMapping.getAllStates()) { @@ -205,21 +225,50 @@ protected void startServices() throws Exception { @Override public void runJob(Properties jobProps, JobListener jobListener) throws JobException { - new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext).call(); + new HelixRetriggeringJobCallable(this, + this.jobCatalog, + this.properties, + jobProps, + jobListener, + this.planningJobLauncherMetrics, + this.helixMetrics, + this.appWorkDir, + this.jobHelixManager, + this.taskDriverHelixManager, + this.jobsMapping, + this.locks, + this.metricContext).call(); } - @Override public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) throws Exception { + @Override + public GobblinHelixJobLauncher buildJobLauncher(Properties jobProps) + throws Exception { Properties combinedProps = new Properties(); combinedProps.putAll(properties); combinedProps.putAll(jobProps); - return new GobblinHelixJobLauncher(combinedProps, this.jobHelixManager, this.appWorkDir, this.metadataTags, this.jobRunningMap, Optional.of(this.helixMetrics)); + return new GobblinHelixJobLauncher(combinedProps, + this.jobHelixManager, + this.appWorkDir, + this.metadataTags, + this.jobRunningMap, + Optional.of(this.helixMetrics)); } public Future scheduleJobImmediately(Properties jobProps, JobListener jobListener) { - HelixRetriggeringJobCallable retriggeringJob = new HelixRetriggeringJobCallable(this, this.jobCatalog, this.properties, jobProps, jobListener, - this.planningJobLauncherMetrics, this.helixMetrics, this.appWorkDir, this.jobHelixManager, - this.taskDriverHelixManager, this.jobsMapping, this.locks, this.metricContext); + HelixRetriggeringJobCallable retriggeringJob = new HelixRetriggeringJobCallable(this, + this.jobCatalog, + this.properties, + jobProps, + jobListener, + this.planningJobLauncherMetrics, + this.helixMetrics, + this.appWorkDir, + this.jobHelixManager, + this.taskDriverHelixManager, + this.jobsMapping, + this.locks, + this.metricContext); final Future future = this.jobExecutor.submit(retriggeringJob); return new Future() { @@ -279,10 +328,12 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobUri); - scheduleJob(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics)); + scheduleJob(jobProps, + new GobblinHelixJobLauncherListener(this.launcherMetrics)); } else { LOGGER.info("No job schedule found, so running job " + jobUri); - this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, new GobblinHelixJobLauncherListener(this.launcherMetrics))); + this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, + new GobblinHelixJobLauncherListener(this.launcherMetrics))); } this.jobStartTimeMap.put(jobUri, Instant.now()); @@ -309,13 +360,15 @@ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobAr } try { - handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); + handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), + updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } try { - handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); + handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(), + updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); } @@ -343,7 +396,9 @@ public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobAr } } - @Subscribe public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) throws InterruptedException { + @Subscribe + public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) + throws InterruptedException { String jobUri = cancelJobArrival.getJoburi(); LOGGER.info("Received cancel for job configuration of job " + jobUri); Optional distributedJobMode; @@ -387,10 +442,11 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t //Workaround for preventing indefinite hangs observed in TaskDriver.getWorkflows() call. Callable> workflowsCallable = () -> HelixUtils.getWorkflowIdsFromJobNames(this.jobHelixManager, - Collections.singletonList(deleteJobArrival.getJobName())); + Collections.singletonList(deleteJobArrival.getJobName())); Retryer> retryer = RetryerBuilder.>newBuilder() .retryIfException() - .withStopStrategy(StopStrategies.stopAfterAttempt(5)).withAttemptTimeLimiter(AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)).build(); + .withStopStrategy(StopStrategies.stopAfterAttempt(5)) + .withAttemptTimeLimiter(AttemptTimeLimiters.fixedTimeLimit(this.helixWorkflowListingTimeoutMillis, TimeUnit.MILLISECONDS)).build(); Map jobNameToWorkflowIdMap; try { jobNameToWorkflowIdMap = retryer.call(workflowsCallable); @@ -402,7 +458,7 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t if (jobNameToWorkflowIdMap.containsKey(deleteJobArrival.getJobName())) { String workflowId = jobNameToWorkflowIdMap.get(deleteJobArrival.getJobName()); boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(jobConfig, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE, - String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); + String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE)); HelixUtils.cancelWorkflow(workflowId, this.jobHelixManager, helixJobStopTimeoutMillis, cancelByDelete); LOGGER.info("Cancelled workflow: {}", deleteJobArrival.getJobName()); //Wait until the cancelled job is complete. @@ -412,6 +468,7 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t } } } + /** * This class is responsible for running non-scheduled jobs. */ @@ -420,7 +477,9 @@ class NonScheduledJobRunner implements Runnable { private final GobblinHelixJobLauncherListener jobListener; private final Long creationTimeInMillis; - public NonScheduledJobRunner(Properties jobProps,GobblinHelixJobLauncherListener jobListener) { + public NonScheduledJobRunner(Properties jobProps, + GobblinHelixJobLauncherListener jobListener) { + this.jobProps = jobProps; this.jobListener = jobListener; this.creationTimeInMillis = System.currentTimeMillis(); @@ -437,4 +496,4 @@ public void run() { } } } -} +} \ No newline at end of file From d75e5224a9afa0b0ef8140f4ad7ea74e06048a6a Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Wed, 14 Jun 2023 14:03:52 -0700 Subject: [PATCH 06/20] [GOBBLIN-1840] Clean up auto format --- .../cluster/GobblinHelixJobScheduler.java | 58 +++++++++---------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 60d122c4ca2..58dea3af551 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -117,12 +117,12 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private ConcurrentHashMap jobStartTimeMap; public GobblinHelixJobScheduler(Config sysConfig, - HelixManager jobHelixManager, - Optional taskDriverHelixManager, - EventBus eventBus, - Path appWorkDir, List> metadataTags, - SchedulerService schedulerService, - MutableJobCatalog jobCatalog) throws Exception { + HelixManager jobHelixManager, + Optional taskDriverHelixManager, + EventBus eventBus, + Path appWorkDir, List> metadataTags, + SchedulerService schedulerService, + MutableJobCatalog jobCatalog) throws Exception { super(ConfigUtils.configToProperties(sysConfig), schedulerService); this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); @@ -136,28 +136,28 @@ public GobblinHelixJobScheduler(Config sysConfig, this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass()); int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig, - ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, - ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); + ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, + ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES); this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler", - this.metricContext, - metricsWindowSizeInMin); + this.metricContext, + metricsWindowSizeInMin); this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor, - this.metricContext, - metricsWindowSizeInMin); + this.metricContext, + metricsWindowSizeInMin); this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties), PathUtils.getRootPath(appWorkDir).toUri(), appWorkDir.toString()); this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler", - this.metricContext, - metricsWindowSizeInMin, this.jobsMapping); + this.metricContext, + metricsWindowSizeInMin, this.jobsMapping); this.helixMetrics = new GobblinHelixMetrics("helixMetricsInJobScheduler", - this.metricContext, - metricsWindowSizeInMin); + this.metricContext, + metricsWindowSizeInMin); this.startServicesCompleted = false; @@ -168,8 +168,8 @@ public GobblinHelixJobScheduler(Config sysConfig, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; this.throttleTimeoutDuration = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); - + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + this.jobStartTimeMap = new ConcurrentHashMap<>(); } @@ -177,9 +177,9 @@ public GobblinHelixJobScheduler(Config sysConfig, @Override public Collection getStandardMetricsCollection() { return ImmutableList.of(this.launcherMetrics, - this.jobSchedulerMetrics, - this.planningJobLauncherMetrics, - this.helixMetrics); + this.jobSchedulerMetrics, + this.planningJobLauncherMetrics, + this.helixMetrics); } @Override @@ -198,9 +198,9 @@ public void scheduleJob(Properties jobProps, JobListener jobListener) throws Job } scheduleJob(jobProps, - jobListener, - Maps.newHashMap(), - GobblinHelixJob.class); + jobListener, + Maps.newHashMap(), + GobblinHelixJob.class); } catch (Exception e) { throw new JobException("Failed to schedule job " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e); @@ -329,11 +329,11 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobUri); scheduleJob(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics)); + new GobblinHelixJobLauncherListener(this.launcherMetrics)); } else { LOGGER.info("No job schedule found, so running job " + jobUri); this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics))); + new GobblinHelixJobLauncherListener(this.launcherMetrics))); } this.jobStartTimeMap.put(jobUri, Instant.now()); @@ -360,7 +360,7 @@ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobAr } try { - handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), + handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); @@ -478,7 +478,7 @@ class NonScheduledJobRunner implements Runnable { private final Long creationTimeInMillis; public NonScheduledJobRunner(Properties jobProps, - GobblinHelixJobLauncherListener jobListener) { + GobblinHelixJobLauncherListener jobListener) { this.jobProps = jobProps; this.jobListener = jobListener; @@ -496,4 +496,4 @@ public void run() { } } } -} \ No newline at end of file +} From 4ddd7c5b8cfab87f9a526c3fcfd7c8abb896cc3f Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Wed, 14 Jun 2023 14:21:01 -0700 Subject: [PATCH 07/20] [GOBBLIN-1840] Clear up empty space --- .../apache/gobblin/cluster/GobblinHelixJobScheduler.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 58dea3af551..ad8aebab552 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -168,8 +168,8 @@ public GobblinHelixJobScheduler(Config sysConfig, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; this.throttleTimeoutDuration = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); - + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + this.jobStartTimeMap = new ConcurrentHashMap<>(); } @@ -360,7 +360,7 @@ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobAr } try { - handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), + handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(), updateJobArrival.getJobConfig())); } catch (Exception je) { LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je); From 0e7ba4dc5dffa3cd594f9a0cbfc8db914147b935 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 15 Jun 2023 11:50:23 -0700 Subject: [PATCH 08/20] [GOBBLIN-1840] Clarify naming standards and simplify repeated codes --- .../cluster/GobblinHelixJobScheduler.java | 22 +- .../cluster/GobblinHelixJobSchedulerTest.java | 263 ++++++------------ 2 files changed, 93 insertions(+), 192 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index ad8aebab552..f8fe50abb63 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -113,8 +113,8 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private boolean startServicesCompleted; private final long helixJobStopTimeoutMillis; - private final Duration throttleTimeoutDuration; - private ConcurrentHashMap jobStartTimeMap; + private final Duration throttleTimeoutDurationSecs; + private ConcurrentHashMap jobNameToStartTimeMap; public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, @@ -167,10 +167,10 @@ public GobblinHelixJobScheduler(Config sysConfig, this.helixWorkflowListingTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; - this.throttleTimeoutDuration = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, + this.throttleTimeoutDurationSecs = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); - this.jobStartTimeMap = new ConcurrentHashMap<>(); + this.jobNameToStartTimeMap = new ConcurrentHashMap<>(); } @@ -336,7 +336,7 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { new GobblinHelixJobLauncherListener(this.launcherMetrics))); } - this.jobStartTimeMap.put(jobUri, Instant.now()); + this.jobNameToStartTimeMap.put(jobUri, Instant.now()); } catch (JobException je) { LOGGER.error("Failed to schedule or run job " + jobUri, je); } @@ -346,15 +346,15 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); String jobName = updateJobArrival.getJobName(); - boolean throttleEnabled = PropertiesUtils.getPropAsBoolean(updateJobArrival.getJobConfig(), + boolean isThrottleEnabled = PropertiesUtils.getPropAsBoolean(updateJobArrival.getJobConfig(), GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY)); - if (throttleEnabled && this.jobStartTimeMap.containsKey(jobName)) { - Instant jobStartTime = this.jobStartTimeMap.get(jobName); - Duration workflowDuration = Duration.between(jobStartTime, Instant.now()); - Duration difference = workflowDuration.minus(throttleTimeoutDuration); - if (difference.isNegative()) { + if (isThrottleEnabled && this.jobNameToStartTimeMap.containsKey(jobName)) { + Instant jobStartTime = this.jobNameToStartTimeMap.get(jobName); + Duration workflowRunningDuration = Duration.between(jobStartTime, Instant.now()); + if (workflowRunningDuration.minus(throttleTimeoutDurationSecs).isNegative()) { + LOGGER.info("Replanning is skipped for job {} ", jobName); return; } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 3578d02d7a6..1c40ad26e07 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -79,8 +79,8 @@ public class GobblinHelixJobSchedulerTest { private final String workflowIdSuffix3 = "_1504201348473"; private Instant beginTime = Instant.ofEpochMilli(0); - private Instant shortPeriod = Instant.ofEpochMilli(1); - private Instant longPeriod = Instant.ofEpochMilli(3600001); + private Instant withinThrottlePeriod = Instant.ofEpochMilli(1); + private Instant exceedsThrottlePeriod = Instant.ofEpochMilli(3600001); private String zkConnectingString; private String helixClusterName; @@ -107,17 +107,11 @@ public void setUp() ConfigValueFactory.fromAnyRef(sourceJsonFile.getAbsolutePath())) .withValue(ConfigurationKeys.JOB_STATE_IN_STATE_STORE, ConfigValueFactory.fromAnyRef("true")).resolve(); - zkConnectingString = baseConfig.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY); - helixClusterName = baseConfig.getString(GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY); + this.zkConnectingString = baseConfig.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY); + this.helixClusterName = baseConfig.getString(GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY); HelixUtils.createGobblinHelixCluster(zkConnectingString, helixClusterName); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - this.closer.register(() -> helixManager.disconnect()); - helixManager.connect(); - this.localFs = FileSystem.getLocal(new Configuration()); this.closer.register(() -> { @@ -140,214 +134,83 @@ public void setUp() this.thread.start(); } + // Time span exceeds throttle timeout, within same workflow, throttle is enabled + // Job will be updated @Test public void testNewJobAndUpdate() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, longPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "NewJobAndUpdate", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); - } + runWorkflowTest(exceedsThrottlePeriod, "UpdateSameWorkflowLongPeriodThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + true, true); } + // Time span is within throttle timeout, within same workflow, throttle is enabled + // Job will not be updated @Test public void testUpdateSameWorkflowShortPeriodThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateSameWorkflowShortPeriodThrottle", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - } + runWorkflowTest(withinThrottlePeriod, "UpdateSameWorkflowShortPeriodThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix1, + true, true); } + // Time span exceeds throttle timeout, within same workflow, throttle is not enabled + // Job will be updated @Test public void testUpdateSameWorkflowLongPeriodNoThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, longPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateSameWorkflowLongPeriodNoThrottle", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); - } + runWorkflowTest(exceedsThrottlePeriod, "UpdateSameWorkflowLongPeriodNoThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + false, true); } + // Time span is within throttle timeout, within same workflow, throttle is not enabled + // Job will be updated @Test public void testUpdateSameWorkflowShortPeriodNoThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateSameWorkflowShortPeriodNoThrottle", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - properties1.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties1.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties1)); - connectAndAssertWorkflowId(workflowIdSuffix2, newJobConfigArrivalEvent, helixManager); - } + runWorkflowTest(withinThrottlePeriod, "UpdateSameWorkflowShortPeriodNoThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + false, true); } + // Time span is within throttle timeout, within different workflow, throttle is enabled + // Job will be updated public void testUpdateDiffWorkflowShortPeriodThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle1", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - final Properties properties2 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowShortPeriodThrottle2", workflowIdSuffix3); - NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = - new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); - properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); - } + runWorkflowTest(withinThrottlePeriod, "UpdateDiffWorkflowShortPeriodThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + true, false); } + // Time span is within throttle timeout, within different workflow, throttle is not enabled + // Job will be updated @Test public void testUpdateDiffWorkflowShortPeriodNoThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, shortPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle1", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - final Properties properties2 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowShortPeriodNoThrottle2", workflowIdSuffix3); - NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = - new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); - properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); - } + runWorkflowTest(withinThrottlePeriod, "UpdateDiffWorkflowShortPeriodNoThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + false, false); } + // Time span exceeds throttle timeout, within different workflow, throttle is enabled + // Job will be updated @Test public void testUpdateDiffWorkflowLongPeriodThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, longPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle1", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - final Properties properties2 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowLongPeriodThrottle2", workflowIdSuffix3); - NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = - new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); - properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "true"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); - } + runWorkflowTest(exceedsThrottlePeriod, "UpdateDiffWorkflowLongPeriodThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + true, false); } + // Time span exceeds throttle timeout, within different workflow, throttle is not enabled + // Job will be updated @Test public void testUpdateDiffWorkflowLongPeriodNoThrottle() throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, longPeriod); - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties1 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle1", workflowIdSuffix1); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties1, workflowIdSuffix2); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(workflowIdSuffix1, newJobConfigArrivalEvent, helixManager); - - final Properties properties2 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, "UpdateDiffWorkflowLongPeriodNoThrottle2", workflowIdSuffix3); - NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = - new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); - properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, "false"); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(workflowIdSuffix3, newJobConfigArrivalEvent2, helixManager); - } + runWorkflowTest(exceedsThrottlePeriod, "UpdateDiffWorkflowLongPeriodNoThrottle", + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + false, false); } private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager) throws Exception { @@ -379,6 +242,7 @@ private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArriv private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { + // endTime is manually set time period that we allow HelixUtils to fetch workflowIdMap before timeout long endTime = System.currentTimeMillis() + 30000; Map workflowIdMap; while (System.currentTimeMillis() < endTime) { @@ -396,6 +260,43 @@ private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, return null; } + private void runWorkflowTest(Instant mockedTime, String jobSuffix, String newJobWorkflowIdSuffix, + String updateWorkflowIdSuffix1, String updateWorkflowIdSuffix2, + String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { + try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { + mocked.when(Instant::now).thenReturn(beginTime, mockedTime); + + // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests + HelixManager helixManager = HelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); + final Properties properties = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, jobSuffix, newJobWorkflowIdSuffix); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties, updateWorkflowIdSuffix1); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(newJobWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + + if (isSameWorkflow) { + properties.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, String.valueOf(isThrottleEnabled)); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties)); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + } + else { + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, jobSuffix + '2', updateWorkflowIdSuffix2); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, String.valueOf(isThrottleEnabled)); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent2, helixManager); + } + } + } @AfterClass public void tearDown() From c449a71b64bbe7aa534639edc6dd699c35e003c3 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 15 Jun 2023 13:34:44 -0700 Subject: [PATCH 09/20] [GOBBLIN-1840] Add Javadoc on GobblinHelixJobSchedulerTest for setting HelixManager as local variable --- .../GobblinClusterConfigurationKeys.java | 4 +- .../cluster/GobblinHelixJobLauncher.java | 2 +- .../cluster/GobblinHelixJobScheduler.java | 69 +++++--- ...bblinThrottleHelixJobLauncherListener.java | 58 ++++++ .../cluster/GobblinHelixJobSchedulerTest.java | 167 +++++++++++------- 5 files changed, 213 insertions(+), 87 deletions(-) create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java index 7e6953df564..ef83ab029a2 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java @@ -17,6 +17,8 @@ package org.apache.gobblin.cluster; +import java.time.Duration; + import org.apache.gobblin.annotation.Alpha; @@ -227,6 +229,6 @@ public class GobblinClusterConfigurationKeys { public static final boolean DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY = false; public static final String HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY = "helix.job.scheduling.throttle.timeout.seconds"; - public static final long DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY = 3600; + public static final long DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY = Duration.ofMinutes(40).getSeconds();; } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java index c0578a99703..a578abb5d07 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java @@ -579,7 +579,7 @@ public Void call() throws Exception { return workUnitFile.toString(); } - private void waitForJobCompletion() throws InterruptedException { + private void waitForJobCompletion() throws Exception { boolean timeoutEnabled = Boolean.parseBoolean( this.jobProps.getProperty(GobblinClusterConfigurationKeys.HELIX_JOB_TIMEOUT_ENABLED_KEY, GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_TIMEOUT_ENABLED)); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index f8fe50abb63..05b2f57fea8 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -18,6 +18,7 @@ package org.apache.gobblin.cluster; import java.io.IOException; +import java.time.Clock; import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; @@ -113,8 +114,10 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private boolean startServicesCompleted; private final long helixJobStopTimeoutMillis; - private final Duration throttleTimeoutDurationSecs; - private ConcurrentHashMap jobNameToStartTimeMap; + private final Duration jobSchedulingThrottleTimeout; + private ConcurrentHashMap jobNameToNextSchedulableTime; + private boolean isThrottleEnabled; + private Clock clock; public GobblinHelixJobScheduler(Config sysConfig, HelixManager jobHelixManager, @@ -123,7 +126,6 @@ public GobblinHelixJobScheduler(Config sysConfig, Path appWorkDir, List> metadataTags, SchedulerService schedulerService, MutableJobCatalog jobCatalog) throws Exception { - super(ConfigUtils.configToProperties(sysConfig), schedulerService); this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); this.jobHelixManager = jobHelixManager; @@ -167,11 +169,27 @@ public GobblinHelixJobScheduler(Config sysConfig, this.helixWorkflowListingTimeoutMillis = ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS, GobblinClusterConfigurationKeys.DEFAULT_HELIX_WORKFLOW_LISTING_TIMEOUT_SECONDS) * 1000; - this.throttleTimeoutDurationSecs = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + this.jobSchedulingThrottleTimeout = Duration.of(ConfigUtils.getLong(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY, + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY), ChronoUnit.SECONDS); + + this.jobNameToNextSchedulableTime = new ConcurrentHashMap<>(); + + this.isThrottleEnabled = ConfigUtils.getBoolean(sysConfig, GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY); - this.jobNameToStartTimeMap = new ConcurrentHashMap<>(); + this.clock = clock; + } + public GobblinHelixJobScheduler(Config sysConfig, + HelixManager jobHelixManager, + Optional taskDriverHelixManager, + EventBus eventBus, + Path appWorkDir, List> metadataTags, + SchedulerService schedulerService, + MutableJobCatalog jobCatalog) throws Exception { + + this(sysConfig, jobHelixManager, taskDriverHelixManager, eventBus, appWorkDir, metadataTags, + schedulerService, jobCatalog, Clock.systemUTC()); } @Override @@ -313,7 +331,7 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec } @Subscribe - public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { + public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { String jobUri = newJobArrival.getJobName(); LOGGER.info("Received new job configuration of job " + jobUri); try { @@ -325,38 +343,35 @@ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { jobProps.setProperty(GobblinClusterConfigurationKeys.JOB_SPEC_URI, jobUri); this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps); - + GobblinHelixJobLauncherListener listener = isThrottleEnabled ? + new GobblinThrottleHelixJobLauncherListener(this.launcherMetrics, jobNameToNextSchedulableTime, + jobSchedulingThrottleTimeout, clock) + : new GobblinHelixJobLauncherListener(this.launcherMetrics); if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobUri); scheduleJob(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics)); + listener); } else { - LOGGER.info("No job schedule found, so running job " + jobUri); + LOGGER.info("No job schedule" + + " found, so running job " + jobUri); this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, - new GobblinHelixJobLauncherListener(this.launcherMetrics))); + listener)); } - this.jobNameToStartTimeMap.put(jobUri, Instant.now()); } catch (JobException je) { LOGGER.error("Failed to schedule or run job " + jobUri, je); } } @Subscribe - public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { + public synchronized void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); String jobName = updateJobArrival.getJobName(); - boolean isThrottleEnabled = PropertiesUtils.getPropAsBoolean(updateJobArrival.getJobConfig(), - GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, - String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY)); - - if (isThrottleEnabled && this.jobNameToStartTimeMap.containsKey(jobName)) { - Instant jobStartTime = this.jobNameToStartTimeMap.get(jobName); - Duration workflowRunningDuration = Duration.between(jobStartTime, Instant.now()); - if (workflowRunningDuration.minus(throttleTimeoutDurationSecs).isNegative()) { - LOGGER.info("Replanning is skipped for job {} ", jobName); - return; - } + + if (this.isThrottleEnabled && + this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)).isAfter(clock.instant())) { + LOGGER.info("Replanning is skipped for job {} ", jobName); + return; } try { @@ -386,7 +401,7 @@ private void waitForJobCompletion(String jobName) { } @Subscribe - public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException { + public synchronized void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException { LOGGER.info("Received delete for job configuration of job " + deleteJobArrival.getJobName()); try { unscheduleJob(deleteJobArrival.getJobName()); @@ -469,6 +484,10 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t } } + public void setThrottleEnabled(boolean throttleEnabled) { + isThrottleEnabled = throttleEnabled; + } + /** * This class is responsible for running non-scheduled jobs. */ diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java new file mode 100644 index 00000000000..4e53b14650c --- /dev/null +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java @@ -0,0 +1,58 @@ +package org.apache.gobblin.cluster; + +import java.time.Clock; +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.gobblin.runtime.JobContext; +import org.apache.gobblin.runtime.JobState; + + +public class GobblinThrottleHelixJobLauncherListener extends GobblinHelixJobLauncherListener { + + public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottleHelixJobLauncherListener.class); + private ConcurrentHashMap jobNameToNextSchedulableTime; + private Duration helixJobSchedulingThrottleTimeout; + private Clock clock; + + GobblinThrottleHelixJobLauncherListener(GobblinHelixJobLauncherMetrics jobLauncherMetrics, + ConcurrentHashMap jobNameToNextSchedulableTime, Duration helixJobSchedulingThrottleTimeout, Clock clock) { + super(jobLauncherMetrics); + this.jobNameToNextSchedulableTime = jobNameToNextSchedulableTime; + this.helixJobSchedulingThrottleTimeout = helixJobSchedulingThrottleTimeout; + this.clock = clock; + } + + @Override + public void onJobPrepare(JobContext jobContext) + throws Exception { + super.onJobPrepare(jobContext); + Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); + LOG.info(jobContext.getJobName() + " finishes onJobPrepare at " + finishTime ); + } + + @Override + public void onJobCompletion(JobContext jobContext) + throws Exception { + super.onJobCompletion(jobContext); + if (jobContext.getJobState().getState() == JobState.RunningState.FAILED) { + jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.ofEpochMilli(0)); + } else { + Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); + LOG.info(jobContext.getJobName() + " finishes onJobCompletion at " + finishTime ); + } + } + + @Override + public void onJobCancellation(JobContext jobContext) + throws Exception { + super.onJobCancellation(jobContext); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.ofEpochMilli(0)); + } +} diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 1c40ad26e07..3784f3d6083 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -21,10 +21,13 @@ import java.io.IOException; import java.net.URL; import java.nio.file.Files; +import java.time.Clock; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Collections; import java.util.Map; import java.util.Properties; +import java.util.Set; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; @@ -33,8 +36,14 @@ import org.apache.helix.HelixManager; import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; +import org.apache.helix.task.TaskDriver; +import org.apache.helix.task.TaskState; +import org.apache.helix.task.WorkflowContext; import org.assertj.core.util.Lists; import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -43,6 +52,7 @@ import org.testng.annotations.Test; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableSet; import com.google.common.eventbus.EventBus; import com.google.common.io.Closer; import com.typesafe.config.Config; @@ -62,10 +72,16 @@ * Unit tests for {@link org.apache.gobblin.cluster.GobblinHelixJobScheduler}. * */ -@Test(groups = {"gobblin.cluster"}) + +/** + * In all test cases, we use GobblinHelixManagerFactory instead of + * HelixManagerFactory, and use HelixManager as a local variable to avoid + * the HelixManager (ZkClient) is not connected error when that's set as + * a global variable across tests. + */ +@Test(groups = {"gobblin.cluster"}, singleThreaded = true) public class GobblinHelixJobSchedulerTest { public final static Logger LOG = LoggerFactory.getLogger(GobblinHelixJobSchedulerTest.class); - private FileSystem localFs; private Path appWorkDir; private final Closer closer = Closer.create(); @@ -76,11 +92,12 @@ public class GobblinHelixJobSchedulerTest { private Thread thread; private final String workflowIdSuffix1 = "_1504201348471"; private final String workflowIdSuffix2 = "_1504201348472"; - private final String workflowIdSuffix3 = "_1504201348473"; private Instant beginTime = Instant.ofEpochMilli(0); - private Instant withinThrottlePeriod = Instant.ofEpochMilli(1); - private Instant exceedsThrottlePeriod = Instant.ofEpochMilli(3600001); + private Instant withinThrottlePeriod = beginTime.plus(1, ChronoUnit.SECONDS); + private Instant exceedsThrottlePeriod = beginTime.plus( + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, + ChronoUnit.SECONDS); private String zkConnectingString; private String helixClusterName; @@ -89,7 +106,7 @@ public class GobblinHelixJobSchedulerTest { public void setUp() throws Exception { TestingServer testingZKServer = this.closer.register(new TestingServer(-1)); - LOG.info("Testing ZK Server listening on: " + testingZKServer.getConnectString()); + System.out.println("Testing ZK Server listening on: " + testingZKServer.getConnectString()); URL url = GobblinHelixJobSchedulerTest.class.getClassLoader() .getResource(GobblinHelixJobSchedulerTest.class.getSimpleName() + ".conf"); @@ -137,10 +154,10 @@ public void setUp() // Time span exceeds throttle timeout, within same workflow, throttle is enabled // Job will be updated @Test - public void testNewJobAndUpdate() + public void testUpdateSameWorkflowLongPeriodThrottle() throws Exception { runWorkflowTest(exceedsThrottlePeriod, "UpdateSameWorkflowLongPeriodThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, true, true); } @@ -150,7 +167,7 @@ public void testNewJobAndUpdate() public void testUpdateSameWorkflowShortPeriodThrottle() throws Exception { runWorkflowTest(withinThrottlePeriod, "UpdateSameWorkflowShortPeriodThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix1, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix1, true, true); } @@ -160,7 +177,7 @@ public void testUpdateSameWorkflowShortPeriodThrottle() public void testUpdateSameWorkflowLongPeriodNoThrottle() throws Exception { runWorkflowTest(exceedsThrottlePeriod, "UpdateSameWorkflowLongPeriodNoThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, false, true); } @@ -170,7 +187,7 @@ public void testUpdateSameWorkflowLongPeriodNoThrottle() public void testUpdateSameWorkflowShortPeriodNoThrottle() throws Exception { runWorkflowTest(withinThrottlePeriod, "UpdateSameWorkflowShortPeriodNoThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, workflowIdSuffix2, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, false, true); } @@ -179,7 +196,7 @@ public void testUpdateSameWorkflowShortPeriodNoThrottle() public void testUpdateDiffWorkflowShortPeriodThrottle() throws Exception { runWorkflowTest(withinThrottlePeriod, "UpdateDiffWorkflowShortPeriodThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, true, false); } @@ -189,7 +206,7 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() public void testUpdateDiffWorkflowShortPeriodNoThrottle() throws Exception { runWorkflowTest(withinThrottlePeriod, "UpdateDiffWorkflowShortPeriodNoThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, false, false); } @@ -199,7 +216,7 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() public void testUpdateDiffWorkflowLongPeriodThrottle() throws Exception { runWorkflowTest(exceedsThrottlePeriod, "UpdateDiffWorkflowLongPeriodThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, true, false); } @@ -209,27 +226,34 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() public void testUpdateDiffWorkflowLongPeriodNoThrottle() throws Exception { runWorkflowTest(exceedsThrottlePeriod, "UpdateDiffWorkflowLongPeriodNoThrottle", - workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix3, workflowIdSuffix3, + workflowIdSuffix1, workflowIdSuffix2, workflowIdSuffix2, false, false); } - private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager) throws Exception { + private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager, boolean isThrottleEnabled, Clock clock) throws Exception { java.nio.file.Path p = Files.createTempDirectory(GobblinHelixJobScheduler.class.getSimpleName()); Config config = ConfigFactory.empty().withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY, ConfigValueFactory.fromAnyRef(p.toString())); SchedulerService schedulerService = new SchedulerService(new Properties()); NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); jobCatalog.startAsync(); - return new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), - new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); + GobblinHelixJobScheduler gobblinHelixJobScheduler; + if (isThrottleEnabled) { + gobblinHelixJobScheduler = new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), + new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog, clock); + } + else { + gobblinHelixJobScheduler = new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), + new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); + } + gobblinHelixJobScheduler.setThrottleEnabled(isThrottleEnabled); + return gobblinHelixJobScheduler; } - private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properties, String suffix) { + private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properties) { properties.setProperty(GobblinClusterConfigurationKeys.CANCEL_RUNNING_JOB_ON_DELETE, "true"); NewJobConfigArrivalEvent newJobConfigArrivalEvent = new NewJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties); - properties.setProperty(ConfigurationKeys.JOB_ID_KEY, - "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + suffix); return newJobConfigArrivalEvent; } @@ -240,7 +264,7 @@ private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArriv Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); } - private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) + private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager) throws Exception { // endTime is manually set time period that we allow HelixUtils to fetch workflowIdMap before timeout long endTime = System.currentTimeMillis() + 30000; @@ -249,52 +273,75 @@ private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, try{ workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, Collections.singletonList(newJobConfigArrivalEvent.getJobName())); - } catch(GobblinHelixUnexpectedStateException e){ + if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { + String workflowId = workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); + TaskDriver taskDriver = new TaskDriver(helixManager); + + Set finalizedStates = + ImmutableSet.of(TaskState.ABORTED, TaskState.STOPPED, TaskState.COMPLETED, TaskState.FAILED, TaskState.TIMED_OUT); + + WorkflowContext context = taskDriver.getWorkflowContext(workflowId); + while (context != null && !finalizedStates.contains(context.getWorkflowState())) { + Thread.sleep(100); + context = taskDriver.getWorkflowContext(workflowId); + } + + return workflowId; + } + } catch(IllegalStateException | GobblinHelixUnexpectedStateException e){ continue; } - if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); - } + Thread.sleep(100); } return null; } - private void runWorkflowTest(Instant mockedTime, String jobSuffix, String newJobWorkflowIdSuffix, - String updateWorkflowIdSuffix1, String updateWorkflowIdSuffix2, - String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { - try (MockedStatic mocked = mockStatic(Instant.class, CALLS_REAL_METHODS)) { - mocked.when(Instant::now).thenReturn(beginTime, mockedTime); - - // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests - HelixManager helixManager = HelixManagerFactory - .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, - zkConnectingString); - GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager); - final Properties properties = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, jobSuffix, newJobWorkflowIdSuffix); - NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties, updateWorkflowIdSuffix1); - jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(newJobWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); - - if (isSameWorkflow) { - properties.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, String.valueOf(isThrottleEnabled)); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties)); - connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); - } - else { - final Properties properties2 = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, jobSuffix + '2', updateWorkflowIdSuffix2); - NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = - new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); - properties2.setProperty(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, String.valueOf(isThrottleEnabled)); - jobScheduler.handleUpdateJobConfigArrival( - new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent2, helixManager); + private void runWorkflowTest(Instant mockedTime, String jobSuffix, + String newJobWorkflowIdSuffix, String updateWorkflowIdSuffix, + String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { + Clock mockClock = Mockito.mock(Clock.class); + when(mockClock.instant()).thenAnswer(new Answer() { + private int count = 0; + @Override + public Instant answer(InvocationOnMock invocation) { + if (count++ == 0) { + return beginTime; + } else { + return mockedTime; + } } + }); + + // Use GobblinHelixManagerFactory instead of HelixManagerFactory to avoid the connection error + // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests + HelixManager helixManager = GobblinHelixManagerFactory + .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, + zkConnectingString); + GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager, isThrottleEnabled, mockClock); + final Properties properties = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, jobSuffix, newJobWorkflowIdSuffix); + NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties); + jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); + connectAndAssertWorkflowId(newJobWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + + if (isSameWorkflow) { + properties.setProperty(ConfigurationKeys.JOB_ID_KEY, + "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + updateWorkflowIdSuffix); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties)); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + } + else { + final Properties properties2 = + GobblinHelixJobLauncherTest.generateJobProperties( + this.baseConfig, jobSuffix + '2', updateWorkflowIdSuffix); + NewJobConfigArrivalEvent newJobConfigArrivalEvent2 = + new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); + jobScheduler.handleUpdateJobConfigArrival( + new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent2, helixManager); } } From 3160d8b43ef477e5ba39dcce4809ce4c38f74490 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 22 Jun 2023 15:20:01 -0700 Subject: [PATCH 10/20] [GOBBLIN-1840] Optimize imports and fix unit test errors --- .../cluster/GobblinHelixJobLauncher.java | 2 +- .../cluster/GobblinHelixJobScheduler.java | 5 +++-- ...blinThrottlingHelixJobLauncherListener.java} | 17 ++++++++++++----- .../cluster/GobblinHelixJobSchedulerTest.java | 7 ++----- 4 files changed, 18 insertions(+), 13 deletions(-) rename gobblin-cluster/src/main/java/org/apache/gobblin/cluster/{GobblinThrottleHelixJobLauncherListener.java => GobblinThrottlingHelixJobLauncherListener.java} (73%) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java index a578abb5d07..c0578a99703 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java @@ -579,7 +579,7 @@ public Void call() throws Exception { return workUnitFile.toString(); } - private void waitForJobCompletion() throws Exception { + private void waitForJobCompletion() throws InterruptedException { boolean timeoutEnabled = Boolean.parseBoolean( this.jobProps.getProperty(GobblinClusterConfigurationKeys.HELIX_JOB_TIMEOUT_ENABLED_KEY, GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_TIMEOUT_ENABLED)); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 05b2f57fea8..b896b13f9ba 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -125,7 +125,8 @@ public GobblinHelixJobScheduler(Config sysConfig, EventBus eventBus, Path appWorkDir, List> metadataTags, SchedulerService schedulerService, - MutableJobCatalog jobCatalog) throws Exception { + MutableJobCatalog jobCatalog, + Clock clock) throws Exception { super(ConfigUtils.configToProperties(sysConfig), schedulerService); this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS)); this.jobHelixManager = jobHelixManager; @@ -344,7 +345,7 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps); GobblinHelixJobLauncherListener listener = isThrottleEnabled ? - new GobblinThrottleHelixJobLauncherListener(this.launcherMetrics, jobNameToNextSchedulableTime, + new GobblinThrottlingHelixJobLauncherListener(this.launcherMetrics, jobNameToNextSchedulableTime, jobSchedulingThrottleTimeout, clock) : new GobblinHelixJobLauncherListener(this.launcherMetrics); if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java similarity index 73% rename from gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java rename to gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 4e53b14650c..b33aca32305 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottleHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -12,14 +12,18 @@ import org.apache.gobblin.runtime.JobState; -public class GobblinThrottleHelixJobLauncherListener extends GobblinHelixJobLauncherListener { +/*** + * When throttle is enabled, this class is used for record jobNameToNextSchedulableTime... + */ - public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottleHelixJobLauncherListener.class); +public class GobblinThrottlingHelixJobLauncherListener extends GobblinHelixJobLauncherListener { + + public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottlingHelixJobLauncherListener.class); private ConcurrentHashMap jobNameToNextSchedulableTime; private Duration helixJobSchedulingThrottleTimeout; private Clock clock; - GobblinThrottleHelixJobLauncherListener(GobblinHelixJobLauncherMetrics jobLauncherMetrics, + public GobblinThrottlingHelixJobLauncherListener(GobblinHelixJobLauncherMetrics jobLauncherMetrics, ConcurrentHashMap jobNameToNextSchedulableTime, Duration helixJobSchedulingThrottleTimeout, Clock clock) { super(jobLauncherMetrics); this.jobNameToNextSchedulableTime = jobNameToNextSchedulableTime; @@ -32,8 +36,10 @@ public void onJobPrepare(JobContext jobContext) throws Exception { super.onJobPrepare(jobContext); Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + // rename finishTime jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); - LOG.info(jobContext.getJobName() + " finishes onJobPrepare at " + finishTime ); + LOG.info(jobContext.getJobName() + " finishes prepare."); + LOG.info(jobContext.getJobName() + " next schedulable time is " + finishTime ); } @Override @@ -45,7 +51,8 @@ public void onJobCompletion(JobContext jobContext) } else { Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); - LOG.info(jobContext.getJobName() + " finishes onJobCompletion at " + finishTime ); + LOG.info(jobContext.getJobName() + " finishes completion."); + LOG.info(jobContext.getJobName() + " next schedulable time is " + finishTime ); } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 3784f3d6083..4c5adda46d8 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -34,13 +34,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerFactory; import org.apache.helix.InstanceType; import org.apache.helix.task.TaskDriver; import org.apache.helix.task.TaskState; import org.apache.helix.task.WorkflowContext; import org.assertj.core.util.Lists; -import org.mockito.MockedStatic; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -65,7 +63,7 @@ import org.apache.gobblin.runtime.job_catalog.NonObservingFSJobCatalog; import org.apache.gobblin.scheduler.SchedulerService; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.when; /** @@ -106,7 +104,7 @@ public class GobblinHelixJobSchedulerTest { public void setUp() throws Exception { TestingServer testingZKServer = this.closer.register(new TestingServer(-1)); - System.out.println("Testing ZK Server listening on: " + testingZKServer.getConnectString()); + LOG.info("Testing ZK Server listening on: " + testingZKServer.getConnectString()); URL url = GobblinHelixJobSchedulerTest.class.getClassLoader() .getResource(GobblinHelixJobSchedulerTest.class.getSimpleName() + ".conf"); @@ -285,7 +283,6 @@ private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, Thread.sleep(100); context = taskDriver.getWorkflowContext(workflowId); } - return workflowId; } } catch(IllegalStateException | GobblinHelixUnexpectedStateException e){ From ae2b58c7dd2c3627ee9e8db8ab98b6774077dd10 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 22 Jun 2023 18:16:59 -0700 Subject: [PATCH 11/20] [GOBBLIN-1840] Rewrite log info and add Javadoc --- .../cluster/GobblinHelixJobScheduler.java | 4 +++- ...linThrottlingHelixJobLauncherListener.java | 23 +++++++++---------- .../cluster/GobblinHelixJobSchedulerTest.java | 16 +++---------- 3 files changed, 17 insertions(+), 26 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index b896b13f9ba..20af92128fe 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -371,7 +371,9 @@ public synchronized void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEven if (this.isThrottleEnabled && this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)).isAfter(clock.instant())) { - LOGGER.info("Replanning is skipped for job {} ", jobName); + LOGGER.info("Replanning is skipped for job {}. Current time is " + + clock.instant() + " and the next schedulable time would be " + + this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)), jobName); return; } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index b33aca32305..6561e51e185 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -12,10 +12,12 @@ import org.apache.gobblin.runtime.JobState; -/*** - * When throttle is enabled, this class is used for record jobNameToNextSchedulableTime... +/** + * A job listener used when {@link GobblinHelixJobLauncher} launches a job. + * In {@link GobblinHelixJobScheduler}, when throttling is enabled, this + * listener would record jobName to next schedulable time to decide whether + * the replanning should be executed or skipped. */ - public class GobblinThrottlingHelixJobLauncherListener extends GobblinHelixJobLauncherListener { public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottlingHelixJobLauncherListener.class); @@ -35,11 +37,9 @@ public GobblinThrottlingHelixJobLauncherListener(GobblinHelixJobLauncherMetrics public void onJobPrepare(JobContext jobContext) throws Exception { super.onJobPrepare(jobContext); - Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); - // rename finishTime - jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); - LOG.info(jobContext.getJobName() + " finishes prepare."); - LOG.info(jobContext.getJobName() + " next schedulable time is " + finishTime ); + Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); + LOG.info(jobContext.getJobName() + " finished prepare. The next schedulable time is " + nextSchedulableTime ); } @Override @@ -49,10 +49,9 @@ public void onJobCompletion(JobContext jobContext) if (jobContext.getJobState().getState() == JobState.RunningState.FAILED) { jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.ofEpochMilli(0)); } else { - Instant finishTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobContext.getJobName(), finishTime); - LOG.info(jobContext.getJobName() + " finishes completion."); - LOG.info(jobContext.getJobName() + " next schedulable time is " + finishTime ); + Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); + LOG.info(jobContext.getJobName() + " finished completion. The next schedulable time is " + nextSchedulableTime ); } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 4c5adda46d8..c5e6a70ab84 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; @@ -40,8 +41,6 @@ import org.apache.helix.task.WorkflowContext; import org.assertj.core.util.Lists; import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -298,17 +297,8 @@ private void runWorkflowTest(Instant mockedTime, String jobSuffix, String newJobWorkflowIdSuffix, String updateWorkflowIdSuffix, String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { Clock mockClock = Mockito.mock(Clock.class); - when(mockClock.instant()).thenAnswer(new Answer() { - private int count = 0; - @Override - public Instant answer(InvocationOnMock invocation) { - if (count++ == 0) { - return beginTime; - } else { - return mockedTime; - } - } - }); + AtomicInteger count = new AtomicInteger(0); + when(mockClock.instant()).thenAnswer(invocation -> count.getAndIncrement() == 0 ? beginTime : mockedTime); // Use GobblinHelixManagerFactory instead of HelixManagerFactory to avoid the connection error // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests From e6ea1950823a2c9b79d3d9a8f2708bf4a65d5118 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Fri, 23 Jun 2023 16:49:01 -0700 Subject: [PATCH 12/20] [GOBBLIN-1840] Remove job status check --- .../cluster/GobblinHelixJobSchedulerTest.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index c5e6a70ab84..77a3fc571fc 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -270,24 +270,12 @@ private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, try{ workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, Collections.singletonList(newJobConfigArrivalEvent.getJobName())); - if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - String workflowId = workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); - TaskDriver taskDriver = new TaskDriver(helixManager); - - Set finalizedStates = - ImmutableSet.of(TaskState.ABORTED, TaskState.STOPPED, TaskState.COMPLETED, TaskState.FAILED, TaskState.TIMED_OUT); - - WorkflowContext context = taskDriver.getWorkflowContext(workflowId); - while (context != null && !finalizedStates.contains(context.getWorkflowState())) { - Thread.sleep(100); - context = taskDriver.getWorkflowContext(workflowId); - } - return workflowId; - } } catch(IllegalStateException | GobblinHelixUnexpectedStateException e){ continue; } - + if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { + return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); + } Thread.sleep(100); } return null; From 6e8358cbe0d4de419dabfd051783fa8e77cecd70 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Mon, 26 Jun 2023 18:48:36 -0700 Subject: [PATCH 13/20] [GOBBLIN-1840] Add log info and change config setting --- .../cluster/GobblinHelixJobScheduler.java | 18 +--- ...linThrottlingHelixJobLauncherListener.java | 15 ++- .../cluster/GobblinHelixJobSchedulerTest.java | 92 ++++++++++--------- .../org.mockito.plugins.MockMaker | 1 - 4 files changed, 67 insertions(+), 59 deletions(-) delete mode 100644 gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 20af92128fe..5e07627eeec 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -350,15 +350,11 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ : new GobblinHelixJobLauncherListener(this.launcherMetrics); if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobUri); - scheduleJob(jobProps, - listener); + scheduleJob(jobProps, listener); } else { - LOGGER.info("No job schedule" - + " found, so running job " + jobUri); - this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, - listener)); + LOGGER.info("No job schedule found, so running job " + jobUri); + this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, listener)); } - } catch (JobException je) { LOGGER.error("Failed to schedule or run job " + jobUri, je); } @@ -369,8 +365,8 @@ public synchronized void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEven LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); String jobName = updateJobArrival.getJobName(); - if (this.isThrottleEnabled && - this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)).isAfter(clock.instant())) { + Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.MIN); + if (this.isThrottleEnabled && clock.instant().isBefore(nextSchedulableTime)) { LOGGER.info("Replanning is skipped for job {}. Current time is " + clock.instant() + " and the next schedulable time would be " + this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)), jobName); @@ -487,10 +483,6 @@ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) t } } - public void setThrottleEnabled(boolean throttleEnabled) { - isThrottleEnabled = throttleEnabled; - } - /** * This class is responsible for running non-scheduled jobs. */ diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 6561e51e185..790512d7dab 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -39,7 +39,7 @@ public void onJobPrepare(JobContext jobContext) super.onJobPrepare(jobContext); Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); - LOG.info(jobContext.getJobName() + " finished prepare. The next schedulable time is " + nextSchedulableTime ); + LOG.info("{} finished preparing. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); } @Override @@ -47,11 +47,14 @@ public void onJobCompletion(JobContext jobContext) throws Exception { super.onJobCompletion(jobContext); if (jobContext.getJobState().getState() == JobState.RunningState.FAILED) { - jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.ofEpochMilli(0)); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.EPOCH); + LOG.info("{} failed. The next schedulable time is {} so that any future schedule attempts will be allowed.", + jobContext.getJobName(), + Instant.EPOCH); } else { Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); - LOG.info(jobContext.getJobName() + " finished completion. The next schedulable time is " + nextSchedulableTime ); + LOG.info("{} is completed. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); } } @@ -59,6 +62,10 @@ public void onJobCompletion(JobContext jobContext) public void onJobCancellation(JobContext jobContext) throws Exception { super.onJobCancellation(jobContext); - jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.ofEpochMilli(0)); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.EPOCH); + LOG.info("{} is cancelled. The next schedulable time is {} so that any future schedule attempts will be allowed.", + jobContext.getJobName(), + Instant.EPOCH); + } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index 77a3fc571fc..bc9bbb1887e 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -27,7 +27,6 @@ import java.util.Collections; import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.apache.curator.test.TestingServer; @@ -36,9 +35,6 @@ import org.apache.hadoop.fs.Path; import org.apache.helix.HelixManager; import org.apache.helix.InstanceType; -import org.apache.helix.task.TaskDriver; -import org.apache.helix.task.TaskState; -import org.apache.helix.task.WorkflowContext; import org.assertj.core.util.Lists; import org.mockito.Mockito; import org.slf4j.Logger; @@ -49,7 +45,6 @@ import org.testng.annotations.Test; import com.google.common.base.Optional; -import com.google.common.collect.ImmutableSet; import com.google.common.eventbus.EventBus; import com.google.common.io.Closer; import com.typesafe.config.Config; @@ -68,13 +63,10 @@ /** * Unit tests for {@link org.apache.gobblin.cluster.GobblinHelixJobScheduler}. * - */ - -/** * In all test cases, we use GobblinHelixManagerFactory instead of - * HelixManagerFactory, and use HelixManager as a local variable to avoid - * the HelixManager (ZkClient) is not connected error when that's set as - * a global variable across tests. + * HelixManagerFactory, and instantiate a local HelixManager per test to + * provide isolation and prevent errors caused by the ZKClient being shared + * (e.g. ZKClient is not connected exceptions). */ @Test(groups = {"gobblin.cluster"}, singleThreaded = true) public class GobblinHelixJobSchedulerTest { @@ -90,9 +82,9 @@ public class GobblinHelixJobSchedulerTest { private final String workflowIdSuffix1 = "_1504201348471"; private final String workflowIdSuffix2 = "_1504201348472"; - private Instant beginTime = Instant.ofEpochMilli(0); - private Instant withinThrottlePeriod = beginTime.plus(1, ChronoUnit.SECONDS); - private Instant exceedsThrottlePeriod = beginTime.plus( + private final Instant beginTime = Instant.EPOCH; + private final Instant withinThrottlePeriod = beginTime.plus(1, ChronoUnit.SECONDS); + private final Instant exceedsThrottlePeriod = beginTime.plus( GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, ChronoUnit.SECONDS); @@ -148,8 +140,11 @@ public void setUp() this.thread.start(); } - // Time span exceeds throttle timeout, within same workflow, throttle is enabled - // Job will be updated + /*** + * Time span exceeds throttle timeout, within same workflow, throttle is enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateSameWorkflowLongPeriodThrottle() throws Exception { @@ -158,8 +153,11 @@ public void testUpdateSameWorkflowLongPeriodThrottle() true, true); } - // Time span is within throttle timeout, within same workflow, throttle is enabled - // Job will not be updated + /*** + * Time span is within throttle timeout, within same workflow, throttle is enabled + * Job will not be updated + * @throws Exception + */ @Test public void testUpdateSameWorkflowShortPeriodThrottle() throws Exception { @@ -168,8 +166,11 @@ public void testUpdateSameWorkflowShortPeriodThrottle() true, true); } - // Time span exceeds throttle timeout, within same workflow, throttle is not enabled - // Job will be updated + /*** + * Time span exceeds throttle timeout, within same workflow, throttle is not enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateSameWorkflowLongPeriodNoThrottle() throws Exception { @@ -178,8 +179,11 @@ public void testUpdateSameWorkflowLongPeriodNoThrottle() false, true); } - // Time span is within throttle timeout, within same workflow, throttle is not enabled - // Job will be updated + /*** + * Time span is within throttle timeout, within same workflow, throttle is not enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateSameWorkflowShortPeriodNoThrottle() throws Exception { @@ -188,8 +192,11 @@ public void testUpdateSameWorkflowShortPeriodNoThrottle() false, true); } - // Time span is within throttle timeout, within different workflow, throttle is enabled - // Job will be updated + /*** + * Time span is within throttle timeout, within different workflow, throttle is enabled + * Job will be updated + * @throws Exception + */ public void testUpdateDiffWorkflowShortPeriodThrottle() throws Exception { runWorkflowTest(withinThrottlePeriod, "UpdateDiffWorkflowShortPeriodThrottle", @@ -197,8 +204,11 @@ public void testUpdateDiffWorkflowShortPeriodThrottle() true, false); } - // Time span is within throttle timeout, within different workflow, throttle is not enabled - // Job will be updated + /*** + * Time span is within throttle timeout, within different workflow, throttle is not enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateDiffWorkflowShortPeriodNoThrottle() throws Exception { @@ -207,8 +217,11 @@ public void testUpdateDiffWorkflowShortPeriodNoThrottle() false, false); } - // Time span exceeds throttle timeout, within different workflow, throttle is enabled - // Job will be updated + /*** + * Time span exceeds throttle timeout, within different workflow, throttle is enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateDiffWorkflowLongPeriodThrottle() throws Exception { @@ -217,8 +230,11 @@ public void testUpdateDiffWorkflowLongPeriodThrottle() true, false); } - // Time span exceeds throttle timeout, within different workflow, throttle is not enabled - // Job will be updated + /*** + * Time span exceeds throttle timeout, within different workflow, throttle is not enabled + * Job will be updated + * @throws Exception + */ @Test public void testUpdateDiffWorkflowLongPeriodNoThrottle() throws Exception { @@ -234,16 +250,10 @@ private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager, b SchedulerService schedulerService = new SchedulerService(new Properties()); NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); jobCatalog.startAsync(); - GobblinHelixJobScheduler gobblinHelixJobScheduler; - if (isThrottleEnabled) { - gobblinHelixJobScheduler = new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), + Config helixJobSchedulerConfig = ConfigFactory.empty().withValue("helix.job.scheduling.throttle.enabled", + ConfigValueFactory.fromAnyRef(isThrottleEnabled)); + GobblinHelixJobScheduler gobblinHelixJobScheduler = new GobblinHelixJobScheduler(helixJobSchedulerConfig, helixManager, java.util.Optional.empty(), new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog, clock); - } - else { - gobblinHelixJobScheduler = new GobblinHelixJobScheduler(ConfigFactory.empty(), helixManager, java.util.Optional.empty(), - new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog); - } - gobblinHelixJobScheduler.setThrottleEnabled(isThrottleEnabled); return gobblinHelixJobScheduler; } @@ -263,14 +273,14 @@ private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArriv private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager) throws Exception { - // endTime is manually set time period that we allow HelixUtils to fetch workflowIdMap before timeout + // Poll helix for up to 30 seconds to fetch until a workflow with a matching job name exists in Helix and then return that workflowID long endTime = System.currentTimeMillis() + 30000; Map workflowIdMap; while (System.currentTimeMillis() < endTime) { try{ workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, Collections.singletonList(newJobConfigArrivalEvent.getJobName())); - } catch(IllegalStateException | GobblinHelixUnexpectedStateException e){ + } catch(GobblinHelixUnexpectedStateException e){ continue; } if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { diff --git a/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index ca6ee9cea8e..00000000000 --- a/gobblin-cluster/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1 +0,0 @@ -mock-maker-inline \ No newline at end of file From a15afd8b23112e362bfd205ca019c06bcaebfcb8 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Mon, 26 Jun 2023 18:58:05 -0700 Subject: [PATCH 14/20] [GOBBLIN-1840] Add @Slf4j in GobblinThrottlingHelixJobLauncherListener --- .../cluster/GobblinThrottlingHelixJobLauncherListener.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 790512d7dab..90456d8a322 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -8,6 +8,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import lombok.extern.slf4j.Slf4j; + import org.apache.gobblin.runtime.JobContext; import org.apache.gobblin.runtime.JobState; @@ -18,6 +20,7 @@ * listener would record jobName to next schedulable time to decide whether * the replanning should be executed or skipped. */ +@Slf4j public class GobblinThrottlingHelixJobLauncherListener extends GobblinHelixJobLauncherListener { public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottlingHelixJobLauncherListener.class); From bbe4a0ba1486b86fe7e2c81056159dfc072f65da Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Tue, 27 Jun 2023 14:28:50 -0700 Subject: [PATCH 15/20] [GOBBLIN-1840] Fix race condition of handleNewJobConfigArrival --- .../cluster/GobblinHelixJobScheduler.java | 26 ++++++++--- ...linThrottlingHelixJobLauncherListener.java | 2 +- .../cluster/GobblinHelixJobSchedulerTest.java | 44 ++++++++++--------- 3 files changed, 45 insertions(+), 27 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 5e07627eeec..e1119a84396 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -335,6 +335,19 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { String jobUri = newJobArrival.getJobName(); LOGGER.info("Received new job configuration of job " + jobUri); + + Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobUri, Instant.EPOCH); + if (this.isThrottleEnabled && clock.instant().isBefore(nextSchedulableTime)) { + LOGGER.info("Adding new job is skipped for job {}. Current time is {} and the next schedulable time would be {}", + jobUri, + clock.instant(), + nextSchedulableTime + ); + return; + } + nextSchedulableTime = clock.instant().plus(jobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobUri, nextSchedulableTime); + try { Properties jobProps = new Properties(); jobProps.putAll(this.commonJobProperties); @@ -357,19 +370,22 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ } } catch (JobException je) { LOGGER.error("Failed to schedule or run job " + jobUri, je); + jobNameToNextSchedulableTime.put(jobUri, Instant.EPOCH); } } @Subscribe public synchronized void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); - String jobName = updateJobArrival.getJobName(); + String jobUri = updateJobArrival.getJobName(); - Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.MIN); + Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobUri, Instant.EPOCH); if (this.isThrottleEnabled && clock.instant().isBefore(nextSchedulableTime)) { - LOGGER.info("Replanning is skipped for job {}. Current time is " - + clock.instant() + " and the next schedulable time would be " - + this.jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.ofEpochMilli(0)), jobName); + LOGGER.info("Replanning is skipped for job {}. Current time is {} and the next schedulable time would be {}", + jobUri, + clock.instant(), + nextSchedulableTime + ); return; } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 90456d8a322..0c4776cdd2f 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -29,7 +29,7 @@ public class GobblinThrottlingHelixJobLauncherListener extends GobblinHelixJobLa private Clock clock; public GobblinThrottlingHelixJobLauncherListener(GobblinHelixJobLauncherMetrics jobLauncherMetrics, - ConcurrentHashMap jobNameToNextSchedulableTime, Duration helixJobSchedulingThrottleTimeout, Clock clock) { + ConcurrentHashMap jobNameToNextSchedulableTime, Duration helixJobSchedulingThrottleTimeout, Clock clock) { super(jobLauncherMetrics); this.jobNameToNextSchedulableTime = jobNameToNextSchedulableTime; this.helixJobSchedulingThrottleTimeout = helixJobSchedulingThrottleTimeout; diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index bc9bbb1887e..c8a530bf436 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -22,12 +22,13 @@ import java.net.URL; import java.nio.file.Files; import java.time.Clock; +import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.Collections; import java.util.Map; import java.util.Properties; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; @@ -83,10 +84,9 @@ public class GobblinHelixJobSchedulerTest { private final String workflowIdSuffix2 = "_1504201348472"; private final Instant beginTime = Instant.EPOCH; - private final Instant withinThrottlePeriod = beginTime.plus(1, ChronoUnit.SECONDS); - private final Instant exceedsThrottlePeriod = beginTime.plus( - GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, - ChronoUnit.SECONDS); + private final Duration withinThrottlePeriod = Duration.of(1, ChronoUnit.SECONDS); + private final Duration exceedsThrottlePeriod = + Duration.of(GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, ChronoUnit.SECONDS); private String zkConnectingString; private String helixClusterName; @@ -250,7 +250,7 @@ private GobblinHelixJobScheduler createJobScheduler(HelixManager helixManager, b SchedulerService schedulerService = new SchedulerService(new Properties()); NonObservingFSJobCatalog jobCatalog = new NonObservingFSJobCatalog(config); jobCatalog.startAsync(); - Config helixJobSchedulerConfig = ConfigFactory.empty().withValue("helix.job.scheduling.throttle.enabled", + Config helixJobSchedulerConfig = ConfigFactory.empty().withValue(GobblinClusterConfigurationKeys.HELIX_JOB_SCHEDULING_THROTTLE_ENABLED_KEY, ConfigValueFactory.fromAnyRef(isThrottleEnabled)); GobblinHelixJobScheduler gobblinHelixJobScheduler = new GobblinHelixJobScheduler(helixJobSchedulerConfig, helixManager, java.util.Optional.empty(), new EventBus(), appWorkDir, Lists.emptyList(), schedulerService, jobCatalog, clock); @@ -264,14 +264,14 @@ private NewJobConfigArrivalEvent createJobConfigArrivalEvent(Properties properti return newJobConfigArrivalEvent; } - private void connectAndAssertWorkflowId(String expectedSuffix, NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager ) throws Exception { + private void connectAndAssertWorkflowId(String expectedSuffix, String jobName, HelixManager helixManager) throws Exception { helixManager.connect(); - String workFlowId = getWorkflowID(newJobConfigArrivalEvent, helixManager); + String workFlowId = getWorkflowID(jobName, helixManager); Assert.assertNotNull(workFlowId); Assert.assertTrue(workFlowId.endsWith(expectedSuffix)); } - private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, HelixManager helixManager) + private String getWorkflowID (String jobName, HelixManager helixManager) throws Exception { // Poll helix for up to 30 seconds to fetch until a workflow with a matching job name exists in Helix and then return that workflowID long endTime = System.currentTimeMillis() + 30000; @@ -279,24 +279,28 @@ private String getWorkflowID (NewJobConfigArrivalEvent newJobConfigArrivalEvent, while (System.currentTimeMillis() < endTime) { try{ workflowIdMap = HelixUtils.getWorkflowIdsFromJobNames(helixManager, - Collections.singletonList(newJobConfigArrivalEvent.getJobName())); + Collections.singletonList(jobName)); } catch(GobblinHelixUnexpectedStateException e){ continue; } - if (workflowIdMap.containsKey(newJobConfigArrivalEvent.getJobName())) { - return workflowIdMap.get(newJobConfigArrivalEvent.getJobName()); + if (workflowIdMap.containsKey(jobName)) { + return workflowIdMap.get(jobName); } Thread.sleep(100); } return null; } - private void runWorkflowTest(Instant mockedTime, String jobSuffix, + private void runWorkflowTest(Duration mockedTime, String jobSuffix, String newJobWorkflowIdSuffix, String updateWorkflowIdSuffix, String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { Clock mockClock = Mockito.mock(Clock.class); - AtomicInteger count = new AtomicInteger(0); - when(mockClock.instant()).thenAnswer(invocation -> count.getAndIncrement() == 0 ? beginTime : mockedTime); + AtomicReference nextInstant = new AtomicReference<>(beginTime); + when(mockClock.instant()).thenAnswer(invocation -> { + Instant currentInstant = nextInstant.get(); + nextInstant.set(currentInstant.plus(mockedTime)); + return currentInstant; + }); // Use GobblinHelixManagerFactory instead of HelixManagerFactory to avoid the connection error // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests @@ -304,19 +308,17 @@ private void runWorkflowTest(Instant mockedTime, String jobSuffix, .getZKHelixManager(helixClusterName, TestHelper.TEST_HELIX_INSTANCE_NAME, InstanceType.CONTROLLER, zkConnectingString); GobblinHelixJobScheduler jobScheduler = createJobScheduler(helixManager, isThrottleEnabled, mockClock); - final Properties properties = - GobblinHelixJobLauncherTest.generateJobProperties( - this.baseConfig, jobSuffix, newJobWorkflowIdSuffix); + final Properties properties = GobblinHelixJobLauncherTest.generateJobProperties(this.baseConfig, jobSuffix, newJobWorkflowIdSuffix); NewJobConfigArrivalEvent newJobConfigArrivalEvent = createJobConfigArrivalEvent(properties); jobScheduler.handleNewJobConfigArrival(newJobConfigArrivalEvent); - connectAndAssertWorkflowId(newJobWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + connectAndAssertWorkflowId(newJobWorkflowIdSuffix, newJobConfigArrivalEvent.getJobName(), helixManager); if (isSameWorkflow) { properties.setProperty(ConfigurationKeys.JOB_ID_KEY, "job_" + properties.getProperty(ConfigurationKeys.JOB_NAME_KEY) + updateWorkflowIdSuffix); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties)); - connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent, helixManager); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent.getJobName(), helixManager); } else { final Properties properties2 = @@ -326,7 +328,7 @@ private void runWorkflowTest(Instant mockedTime, String jobSuffix, new NewJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2); jobScheduler.handleUpdateJobConfigArrival( new UpdateJobConfigArrivalEvent(properties2.getProperty(ConfigurationKeys.JOB_NAME_KEY), properties2)); - connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent2, helixManager); + connectAndAssertWorkflowId(assertUpdateWorkflowIdSuffix, newJobConfigArrivalEvent2.getJobName(), helixManager); } } From 701881eefb41675c76703bf801722e9a2a18d04e Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Tue, 27 Jun 2023 15:50:27 -0700 Subject: [PATCH 16/20] [GOBBLIN-1840] Improve mockClock mechanism --- .../cluster/GobblinHelixJobScheduler.java | 58 +++++++++++-------- .../cluster/GobblinHelixJobSchedulerTest.java | 8 +-- 2 files changed, 37 insertions(+), 29 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index e1119a84396..caa891a9a23 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -235,9 +235,9 @@ protected void startServices() throws Exception { if (cleanAllDistJobs) { for (org.apache.gobblin.configuration.State state : this.jobsMapping.getAllStates()) { - String jobUri = state.getId(); - LOGGER.info("Delete mapping for job " + jobUri); - this.jobsMapping.deleteMapping(jobUri); + String jobName = state.getId(); + LOGGER.info("Delete mapping for job " + jobName); + this.jobsMapping.deleteMapping(jobName); } } } @@ -333,20 +333,20 @@ public Object get(long timeout, TimeUnit unit) throws InterruptedException, Exec @Subscribe public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) { - String jobUri = newJobArrival.getJobName(); - LOGGER.info("Received new job configuration of job " + jobUri); + String jobName = newJobArrival.getJobName(); + LOGGER.info("Received new job configuration of job " + jobName); - Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobUri, Instant.EPOCH); + Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.EPOCH); if (this.isThrottleEnabled && clock.instant().isBefore(nextSchedulableTime)) { LOGGER.info("Adding new job is skipped for job {}. Current time is {} and the next schedulable time would be {}", - jobUri, + jobName, clock.instant(), nextSchedulableTime ); return; } nextSchedulableTime = clock.instant().plus(jobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobUri, nextSchedulableTime); + jobNameToNextSchedulableTime.put(jobName, nextSchedulableTime); try { Properties jobProps = new Properties(); @@ -354,7 +354,7 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ jobProps.putAll(newJobArrival.getJobConfig()); // set uri so that we can delete this job later - jobProps.setProperty(GobblinClusterConfigurationKeys.JOB_SPEC_URI, jobUri); + jobProps.setProperty(GobblinClusterConfigurationKeys.JOB_SPEC_URI, jobName); this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps); GobblinHelixJobLauncherListener listener = isThrottleEnabled ? @@ -362,27 +362,30 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ jobSchedulingThrottleTimeout, clock) : new GobblinHelixJobLauncherListener(this.launcherMetrics); if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { - LOGGER.info("Scheduling job " + jobUri); + LOGGER.info("Scheduling job " + jobName); scheduleJob(jobProps, listener); } else { - LOGGER.info("No job schedule found, so running job " + jobUri); + LOGGER.info("No job schedule found, so running job " + jobName); this.jobExecutor.execute(new NonScheduledJobRunner(jobProps, listener)); } } catch (JobException je) { - LOGGER.error("Failed to schedule or run job " + jobUri, je); - jobNameToNextSchedulableTime.put(jobUri, Instant.EPOCH); + LOGGER.error("Failed to schedule or run job {} . Reset the next scheduable time to {}", + jobName, + Instant.EPOCH, + je); + jobNameToNextSchedulableTime.put(jobName, Instant.EPOCH); } } @Subscribe public synchronized void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) { LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName()); - String jobUri = updateJobArrival.getJobName(); + String jobName = updateJobArrival.getJobName(); - Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobUri, Instant.EPOCH); + Instant nextSchedulableTime = jobNameToNextSchedulableTime.getOrDefault(jobName, Instant.EPOCH); if (this.isThrottleEnabled && clock.instant().isBefore(nextSchedulableTime)) { LOGGER.info("Replanning is skipped for job {}. Current time is {} and the next schedulable time would be {}", - jobUri, + jobName, clock.instant(), nextSchedulableTime ); @@ -415,6 +418,15 @@ private void waitForJobCompletion(String jobName) { } } + /*** + * Deleting a workflow with throttling enabled means that the next + * schedulable time for the workflow will remain unchanged. + * Note: In such case, it is required to wait until the throttle + * timeout period elapses before the workflow can be rescheduled. + * + * @param deleteJobArrival + * @throws InterruptedException + */ @Subscribe public synchronized void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException { LOGGER.info("Received delete for job configuration of job " + deleteJobArrival.getJobName()); @@ -429,8 +441,8 @@ public synchronized void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEven @Subscribe public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival) throws InterruptedException { - String jobUri = cancelJobArrival.getJoburi(); - LOGGER.info("Received cancel for job configuration of job " + jobUri); + String jobName = cancelJobArrival.getJoburi(); + LOGGER.info("Received cancel for job configuration of job " + jobName); Optional distributedJobMode; Optional planningJob = Optional.empty(); Optional actualJob = Optional.empty(); @@ -440,14 +452,14 @@ public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobAr this.jobSchedulerMetrics.numCancellationStart.incrementAndGet(); try { - distributedJobMode = this.jobsMapping.getDistributedJobMode(jobUri); + distributedJobMode = this.jobsMapping.getDistributedJobMode(jobName); if (distributedJobMode.isPresent() && Boolean.parseBoolean(distributedJobMode.get())) { - planningJob = this.jobsMapping.getPlanningJobId(jobUri); + planningJob = this.jobsMapping.getPlanningJobId(jobName); } else { - actualJob = this.jobsMapping.getActualJobId(jobUri); + actualJob = this.jobsMapping.getActualJobId(jobName); } } catch (IOException e) { - LOGGER.warn("jobsMapping could not be retrieved for job {}", jobUri); + LOGGER.warn("jobsMapping could not be retrieved for job {}", jobName); return; } @@ -522,7 +534,7 @@ public void run() { GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis()); GobblinHelixJobScheduler.this.runJob(this.jobProps, this.jobListener); } catch (JobException je) { - LOGGER.error("Failed to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); + LOGGER.error("Failed to schedule or run job to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); } } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index c8a530bf436..e19f45c9af8 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -291,16 +291,12 @@ private String getWorkflowID (String jobName, HelixManager helixManager) return null; } - private void runWorkflowTest(Duration mockedTime, String jobSuffix, + private void runWorkflowTest(Duration mockedPeriod, String jobSuffix, String newJobWorkflowIdSuffix, String updateWorkflowIdSuffix, String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { Clock mockClock = Mockito.mock(Clock.class); AtomicReference nextInstant = new AtomicReference<>(beginTime); - when(mockClock.instant()).thenAnswer(invocation -> { - Instant currentInstant = nextInstant.get(); - nextInstant.set(currentInstant.plus(mockedTime)); - return currentInstant; - }); + when(mockClock.instant()).thenAnswer(invocation -> nextInstant.getAndAccumulate(nextInstant.get(), (currentInstant, x) -> currentInstant.plus(mockedPeriod))); // Use GobblinHelixManagerFactory instead of HelixManagerFactory to avoid the connection error // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests From cfdc115c94f9a09c64019336b7063f8b32b675f3 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Tue, 27 Jun 2023 16:45:16 -0700 Subject: [PATCH 17/20] [GOBBLIN-1840] Address comments --- .../gobblin/cluster/GobblinHelixJobScheduler.java | 12 +++++++++++- .../GobblinThrottlingHelixJobLauncherListener.java | 13 +++++++++---- .../cluster/GobblinHelixJobSchedulerTest.java | 8 ++++---- 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index caa891a9a23..091d2104bf8 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -114,6 +114,16 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe private boolean startServicesCompleted; private final long helixJobStopTimeoutMillis; + + /** + * The throttling timeout prevents helix workflows with the same job name / URI from being submitted + * more than once within the timeout period. This timeout is not reset by deletes / cancels, meaning that + * if you delete a workflow within the timeout period, you cannot reschedule until the timeout period is complete. + * However, if there is an error when launching the job, you can immediately reschedule the flow.

+ * + * NOTE: This throttle timeout period starts when the job launcher thread picks up the runnable. Meaning that the + * time it takes to submit to Helix and start running the flow is also included as part of the timeout period + */ private final Duration jobSchedulingThrottleTimeout; private ConcurrentHashMap jobNameToNextSchedulableTime; private boolean isThrottleEnabled; @@ -534,7 +544,7 @@ public void run() { GobblinHelixJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis()); GobblinHelixJobScheduler.this.runJob(this.jobProps, this.jobListener); } catch (JobException je) { - LOGGER.error("Failed to schedule or run job to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); + LOGGER.error("Failed to schedule or run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je); } } } diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 0c4776cdd2f..33a8b0318b9 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -45,6 +45,15 @@ public void onJobPrepare(JobContext jobContext) LOG.info("{} finished preparing. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); } + @Override + public void onJobStart(JobContext jobContext) + throws Exception { + super.onJobStart(jobContext); + Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); + LOG.info("{} has started. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); + } + @Override public void onJobCompletion(JobContext jobContext) throws Exception { @@ -54,10 +63,6 @@ public void onJobCompletion(JobContext jobContext) LOG.info("{} failed. The next schedulable time is {} so that any future schedule attempts will be allowed.", jobContext.getJobName(), Instant.EPOCH); - } else { - Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); - LOG.info("{} is completed. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); } } diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java index e19f45c9af8..e21c7e73874 100644 --- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java +++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobSchedulerTest.java @@ -85,8 +85,8 @@ public class GobblinHelixJobSchedulerTest { private final Instant beginTime = Instant.EPOCH; private final Duration withinThrottlePeriod = Duration.of(1, ChronoUnit.SECONDS); - private final Duration exceedsThrottlePeriod = - Duration.of(GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, ChronoUnit.SECONDS); + private final Duration exceedsThrottlePeriod = Duration.of( + GobblinClusterConfigurationKeys.DEFAULT_HELIX_JOB_SCHEDULING_THROTTLE_TIMEOUT_SECONDS_KEY + 1, ChronoUnit.SECONDS); private String zkConnectingString; private String helixClusterName; @@ -291,12 +291,12 @@ private String getWorkflowID (String jobName, HelixManager helixManager) return null; } - private void runWorkflowTest(Duration mockedPeriod, String jobSuffix, + private void runWorkflowTest(Duration mockStepAmountTime, String jobSuffix, String newJobWorkflowIdSuffix, String updateWorkflowIdSuffix, String assertUpdateWorkflowIdSuffix, boolean isThrottleEnabled, boolean isSameWorkflow) throws Exception { Clock mockClock = Mockito.mock(Clock.class); AtomicReference nextInstant = new AtomicReference<>(beginTime); - when(mockClock.instant()).thenAnswer(invocation -> nextInstant.getAndAccumulate(nextInstant.get(), (currentInstant, x) -> currentInstant.plus(mockedPeriod))); + when(mockClock.instant()).thenAnswer(invocation -> nextInstant.getAndAccumulate(null, (currentInstant, x) -> currentInstant.plus(mockStepAmountTime))); // Use GobblinHelixManagerFactory instead of HelixManagerFactory to avoid the connection error // helixManager is set to local variable to avoid the HelixManager (ZkClient) is not connected error across tests From b5984557074a2dca271b044b4f8b34763fc196cb Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Wed, 28 Jun 2023 17:38:59 -0700 Subject: [PATCH 18/20] [GOBBLIN-1840] Only put entry in jobNameToNextSchedulableTime when throttle is enabled --- .../gobblin/cluster/GobblinHelixJobScheduler.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index 091d2104bf8..d81d77a35f1 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -355,8 +355,11 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ ); return; } - nextSchedulableTime = clock.instant().plus(jobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobName, nextSchedulableTime); + + if (isThrottleEnabled) { + nextSchedulableTime = clock.instant().plus(jobSchedulingThrottleTimeout); + jobNameToNextSchedulableTime.put(jobName, nextSchedulableTime); + } try { Properties jobProps = new Properties(); @@ -383,7 +386,9 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ jobName, Instant.EPOCH, je); - jobNameToNextSchedulableTime.put(jobName, Instant.EPOCH); + if (isThrottleEnabled) { + jobNameToNextSchedulableTime.put(jobName, Instant.EPOCH); + } } } From 32ea7ed9f9770275b4bfcc95d36cd543dd3b8483 Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 29 Jun 2023 10:21:05 -0700 Subject: [PATCH 19/20] [GOBBLIN-1840] Remove extra schedulable time updates --- .../cluster/GobblinHelixJobScheduler.java | 3 +-- ...linThrottlingHelixJobLauncherListener.java | 25 +------------------ 2 files changed, 2 insertions(+), 26 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java index d81d77a35f1..d30554d2bb3 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java @@ -371,8 +371,7 @@ public synchronized void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJ this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps); GobblinHelixJobLauncherListener listener = isThrottleEnabled ? - new GobblinThrottlingHelixJobLauncherListener(this.launcherMetrics, jobNameToNextSchedulableTime, - jobSchedulingThrottleTimeout, clock) + new GobblinThrottlingHelixJobLauncherListener(this.launcherMetrics, jobNameToNextSchedulableTime) : new GobblinHelixJobLauncherListener(this.launcherMetrics); if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) { LOGGER.info("Scheduling job " + jobName); diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index 33a8b0318b9..ddd76b51309 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -25,33 +25,11 @@ public class GobblinThrottlingHelixJobLauncherListener extends GobblinHelixJobLa public final static Logger LOG = LoggerFactory.getLogger(GobblinThrottlingHelixJobLauncherListener.class); private ConcurrentHashMap jobNameToNextSchedulableTime; - private Duration helixJobSchedulingThrottleTimeout; - private Clock clock; public GobblinThrottlingHelixJobLauncherListener(GobblinHelixJobLauncherMetrics jobLauncherMetrics, - ConcurrentHashMap jobNameToNextSchedulableTime, Duration helixJobSchedulingThrottleTimeout, Clock clock) { + ConcurrentHashMap jobNameToNextSchedulableTime) { super(jobLauncherMetrics); this.jobNameToNextSchedulableTime = jobNameToNextSchedulableTime; - this.helixJobSchedulingThrottleTimeout = helixJobSchedulingThrottleTimeout; - this.clock = clock; - } - - @Override - public void onJobPrepare(JobContext jobContext) - throws Exception { - super.onJobPrepare(jobContext); - Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); - LOG.info("{} finished preparing. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); - } - - @Override - public void onJobStart(JobContext jobContext) - throws Exception { - super.onJobStart(jobContext); - Instant nextSchedulableTime = clock.instant().plus(helixJobSchedulingThrottleTimeout); - jobNameToNextSchedulableTime.put(jobContext.getJobName(), nextSchedulableTime); - LOG.info("{} has started. The next schedulable time is {}", jobContext.getJobName(), nextSchedulableTime); } @Override @@ -74,6 +52,5 @@ public void onJobCancellation(JobContext jobContext) LOG.info("{} is cancelled. The next schedulable time is {} so that any future schedule attempts will be allowed.", jobContext.getJobName(), Instant.EPOCH); - } } From 249621209d3ddbf4de3302d3dd35e11da80dd7fc Mon Sep 17 00:00:00 2001 From: Peiyingy Date: Thu, 29 Jun 2023 13:29:45 -0700 Subject: [PATCH 20/20] [GOBBLIN-1840] Fix checkstyle problems --- ...linThrottlingHelixJobLauncherListener.java | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java index ddd76b51309..ec00be19bd1 100644 --- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java +++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinThrottlingHelixJobLauncherListener.java @@ -1,7 +1,22 @@ +/* + * 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.cluster; -import java.time.Clock; -import java.time.Duration; import java.time.Instant; import java.util.concurrent.ConcurrentHashMap; @@ -39,8 +54,7 @@ public void onJobCompletion(JobContext jobContext) if (jobContext.getJobState().getState() == JobState.RunningState.FAILED) { jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.EPOCH); LOG.info("{} failed. The next schedulable time is {} so that any future schedule attempts will be allowed.", - jobContext.getJobName(), - Instant.EPOCH); + jobContext.getJobName(), Instant.EPOCH); } } @@ -50,7 +64,6 @@ public void onJobCancellation(JobContext jobContext) super.onJobCancellation(jobContext); jobNameToNextSchedulableTime.put(jobContext.getJobName(), Instant.EPOCH); LOG.info("{} is cancelled. The next schedulable time is {} so that any future schedule attempts will be allowed.", - jobContext.getJobName(), - Instant.EPOCH); + jobContext.getJobName(), Instant.EPOCH); } }