From 0ca2cb74d12ab20180f6303be0d794857cb94ed6 Mon Sep 17 00:00:00 2001 From: Agam Pal Singh Date: Mon, 8 Dec 2025 14:41:25 +0530 Subject: [PATCH 01/13] independent memory and container configuration for temporal stages --- conf/yarn/application.conf | 20 ++ .../GobblinTemporalConfigurationKeys.java | 28 ++ .../cluster/AbstractTemporalWorker.java | 18 +- .../cluster/GobblinTemporalTaskRunner.java | 7 +- .../temporal/ddm/activity/ActivityType.java | 20 ++ .../RecommendScalingForWorkUnits.java | 6 +- ...tractRecommendScalingForWorkUnitsImpl.java | 84 +++++- .../ddm/worker/DiscoveryCommitWorker.java | 100 +++++++ .../temporal/ddm/worker/DiscoveryWorker.java | 0 .../temporal/ddm/worker/ExecutionWorker.java | 87 ++++++ .../ddm/worker/WorkFulfillmentWorker.java | 18 +- .../temporal/ddm/workflow/WorkflowStage.java | 111 ++++++++ .../workflow/impl/CommitStepWorkflowImpl.java | 7 +- .../impl/ExecuteGobblinWorkflowImpl.java | 14 +- ...tingExecOfProcessWorkUnitWorkflowImpl.java | 6 +- .../impl/ProcessWorkUnitsWorkflowImpl.java | 2 + .../temporal/dynamic/WorkforcePlan.java | 9 +- .../yarn/DynamicScalingYarnService.java | 249 +++++++++++++++++- .../gobblin/temporal/yarn/YarnService.java | 7 + 19 files changed, 749 insertions(+), 44 deletions(-) create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java create mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java diff --git a/conf/yarn/application.conf b/conf/yarn/application.conf index b31a731ac8e..61c27f09125 100644 --- a/conf/yarn/application.conf +++ b/conf/yarn/application.conf @@ -95,3 +95,23 @@ admin.server.enabled=false # job history store ( WARN [GobblinYarnAppLauncher] NOT starting the admin UI because the job execution info server is NOT enabled ) job.execinfo.server.enabled=false job.history.store.enabled=false + + +# Work Discovery stage +gobblin.temporal.stage.workDiscovery.memory.mb=16384 +gobblin.temporal.stage.workDiscovery.oom.memory.multiplier=2 +gobblin.temporal.stage.workDiscovery.oom.max.memory.mb=131072 + +# Work Execution stage +gobblin.temporal.stage.workExecution.memory.mb=8192 +gobblin.temporal.stage.workExecution.oom.memory.multiplier=2 +gobblin.temporal.stage.workExecution.oom.max.memory.mb=65536 + +# Commit stage +gobblin.temporal.stage.commit.memory.mb=4096 +gobblin.temporal.stage.commit.oom.memory.multiplier=2 +gobblin.temporal.stage.commit.oom.max.memory.mb=32768 + +gobblin.temporal.discovery.task.queue.name=GobblinTemporalDiscoveryQueue +gobblin.temporal.execution.task.queue.name=GobblinTemporalExecutionQueue +gobblin.temporal.commit.task.queue.name=GobblinTemporalCommitQueue diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index cdd5728fdc8..0326b317b6d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -37,6 +37,13 @@ public interface GobblinTemporalConfigurationKeys { String GOBBLIN_TEMPORAL_TASK_QUEUE = PREFIX + "task.queue.name"; String DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue"; + + // Stage-specific task queues for worker specialization + // Discovery and Commit share the same queue (lightweight operations) + String DISCOVERY_COMMIT_TASK_QUEUE = PREFIX + "discovery.commit.task.queue.name"; + String DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE = "GobblinTemporalDiscoveryCommitQueue"; + String EXECUTION_TASK_QUEUE = PREFIX + "execution.task.queue.name"; + String DEFAULT_EXECUTION_TASK_QUEUE = "GobblinTemporalExecutionQueue"; String GOBBLIN_TEMPORAL_JOB_LAUNCHER_PREFIX = PREFIX + "job.launcher."; String GOBBLIN_TEMPORAL_JOB_LAUNCHER_CLASS = GOBBLIN_TEMPORAL_JOB_LAUNCHER_PREFIX + "class"; String DEFAULT_GOBBLIN_TEMPORAL_JOB_LAUNCHER_CLASS = HelloWorldJobLauncher.class.getName(); @@ -130,4 +137,25 @@ public interface GobblinTemporalConfigurationKeys { String TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = TEMPORAL_ACTIVITY_RETRY_OPTIONS + "maximum.attempts"; int DEFAULT_TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = 4; + /** + * Stage-specific resource configuration + * Allows independent memory and OOM configuration per workflow stage + */ + String STAGE_SPECIFIC_PREFIX = PREFIX + "stage."; + + // Discovery/Commit stage configuration (shared by DiscoveryCommitWorker) + // Handles both work discovery and commit activities in the same container + String DISCOVERY_COMMIT_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "discoveryCommit.memory.mb"; + String DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER = STAGE_SPECIFIC_PREFIX + "discoveryCommit.oom.memory.multiplier"; + int DEFAULT_DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER = 2; + String DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "discoveryCommit.oom.max.memory.mb"; + int DEFAULT_DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB = 32768; // 32GB (lightweight operations) + + // Work Execution stage configuration + String WORK_EXECUTION_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.memory.mb"; + String WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = STAGE_SPECIFIC_PREFIX + "workExecution.oom.memory.multiplier"; + int DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = 2; + String WORK_EXECUTION_OOM_MAX_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.oom.max.memory.mb"; + int DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB = 65536; // 64GB (memory-intensive operations) + } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java index 8ab428c4189..26f1e68fb49 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java @@ -33,16 +33,12 @@ /** Basic boilerplate for a {@link TemporalWorker} to register its activity and workflow capabilities and listen on a particular queue */ public abstract class AbstractTemporalWorker implements TemporalWorker { private final WorkflowClient workflowClient; - private final String queueName; private final WorkerFactory workerFactory; - private final Config config; + protected final Config config; public AbstractTemporalWorker(Config cfg, WorkflowClient client) { config = cfg; workflowClient = client; - queueName = ConfigUtils.getString(cfg, - GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, - GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE); // Create a Worker factory that can be used to create Workers that poll specific Task Queues. workerFactory = WorkerFactory.newInstance(workflowClient); @@ -52,7 +48,7 @@ public AbstractTemporalWorker(Config cfg, WorkflowClient client) { @Override public void start() { - Worker worker = workerFactory.newWorker(queueName, createWorkerOptions()); + Worker worker = workerFactory.newWorker(getTaskQueue(), createWorkerOptions()); // This Worker hosts both Workflow and Activity implementations. // Workflows are stateful, so you need to supply a type to create instances. worker.registerWorkflowImplementationTypes(getWorkflowImplClasses()); @@ -77,6 +73,16 @@ protected WorkerOptions createWorkerOptions() { /** @return activity instances; NOTE: activities must be stateless and thread-safe, so a shared instance is used. */ protected abstract Object[] getActivityImplInstances(); + /** + * @return the task queue name this worker should poll from. + * Subclasses can override this to specify a custom task queue. + */ + protected String getTaskQueue() { + return ConfigUtils.getString(config, + GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE); + } + private final void stashWorkerConfig(Config cfg) { // stash to associate with... WorkerConfig.forWorker(this.getClass(), cfg); // the worker itself diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java index c8091068a9a..3621eb3edb0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java @@ -250,8 +250,11 @@ private TemporalWorker initiateWorker() throws Exception { WorkflowClient client = TemporalWorkflowClientFactory.createClientInstance( managedWorkflowServiceStubs.getWorkflowServiceStubs(), namespace); - String workerClassName = ConfigUtils.getString(clusterConfig, - GobblinTemporalConfigurationKeys.WORKER_CLASS, GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS); + // Read worker class from system property (set by YarnService when launching containers) + // Fallback to config for non-YARN deployments + String workerClassName = System.getProperty(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigUtils.getString(clusterConfig, GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); logger.info("Creating worker - class: '{}'", workerClassName); Config workerConfig = clusterConfig; TemporalWorker worker = GobblinConstructorUtils.invokeLongestConstructor( diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java index 22126b2d0fc..b62731c6caa 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java @@ -72,6 +72,26 @@ public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbe .build(); } + /** + * Build activity options with a specific task queue for stage-specific routing. + * @param props job properties + * @param setHeartbeatTimeout whether to set heartbeat timeout + * @param taskQueue the task queue to route this activity to + * @return ActivityOptions with task queue set + */ + public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbeatTimeout, String taskQueue) { + ActivityOptions.Builder builder = ActivityOptions.newBuilder() + .setStartToCloseTimeout(getStartToCloseTimeout(props)) + .setRetryOptions(buildRetryOptions(props)) + .setTaskQueue(taskQueue); // Route to specific queue + + if (setHeartbeatTimeout) { + builder.setHeartbeatTimeout(getHeartbeatTimeout(props)); + } + + return builder.build(); + } + private ActivityOptions buildActivityOptionsWithoutHeartBeatTimeout(Properties props) { return ActivityOptions.newBuilder() .setStartToCloseTimeout(getStartToCloseTimeout(props)) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java index f3715a0d98e..696a4249ab7 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.gobblin.temporal.ddm.activity; import java.util.List; @@ -26,13 +25,13 @@ import org.apache.gobblin.source.workunit.WorkUnit; import org.apache.gobblin.temporal.ddm.work.TimeBudget; import org.apache.gobblin.temporal.ddm.work.WorkUnitsSizeSummary; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.dynamic.ScalingDirective; /** * Activity to suggest the Dynamic Scaling warranted to complete processing of some amount of {@link org.apache.gobblin.source.workunit.WorkUnit}s - * within {@link TimeBudget}, through a combination of Workforce auto-scaling and Worker right-sizing. * * As with all {@link ActivityInterface}s, this is stateless, so the {@link ScalingDirective}(s) returned "stand alone", presuming nothing of current * {@link org.apache.gobblin.temporal.dynamic.WorkforceStaffing}. It thus falls to the caller to coordinate whether to apply the directive(s) as-is, @@ -49,8 +48,9 @@ public interface RecommendScalingForWorkUnits { * @param sourceClass contextualizes the `WorkUnitsSizeSummary` and should name a {@link org.apache.gobblin.source.Source} * @param timeBudget the remaining target duration for processing the summarized `WorkUnit`s * @param jobProps all job props, to either guide the recommendation or better contextualize the nature of the `remainingWork` + * @param stage the workflow stage for which scaling is being recommended (e.g., WORK_EXECUTION) * @return the {@link ScalingDirective}s to process the summarized {@link WorkUnit}s within {@link TimeBudget} */ @ActivityMethod - List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps); + List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps, WorkflowStage stage); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java index a0d3fd11e55..dd11f73cf39 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java @@ -25,13 +25,16 @@ import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.runtime.JobState; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.ddm.activity.RecommendScalingForWorkUnits; import org.apache.gobblin.temporal.ddm.work.TimeBudget; import org.apache.gobblin.temporal.ddm.work.WorkUnitsSizeSummary; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.dynamic.ProfileDerivation; import org.apache.gobblin.temporal.dynamic.ProfileOverlay; import org.apache.gobblin.temporal.dynamic.ScalingDirective; import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; /** @@ -45,33 +48,92 @@ public abstract class AbstractRecommendScalingForWorkUnitsImpl implements Recomm public static final String DEFAULT_PROFILE_DERIVATION_NAME = "workUnitsProc"; @Override - public List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps) { + public List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps, WorkflowStage stage) { // NOTE: no attempt to determine the current scaling - per `RecommendScalingForWorkUnits` javadoc, the `ScalingDirective`(s) returned must "stand alone", // presuming nothing of the current `WorkforcePlan`'s `WorkforceStaffing` JobState jobState = new JobState(jobProps); ScalingDirective procWUsWorkerScaling = new ScalingDirective( - calcProfileDerivationName(jobState), + calcProfileDerivationName(jobState, stage), calcDerivationSetPoint(remainingWork, sourceClass, timeBudget, jobState), System.currentTimeMillis(), - Optional.of(calcProfileDerivation(calcBasisProfileName(jobState), remainingWork, sourceClass, jobState)) + Optional.of(calcProfileDerivation(calcBasisProfileName(jobState, stage), remainingWork, sourceClass, jobState, stage)) ); - log.info("Recommended re-scaling to process work units: {}", procWUsWorkerScaling); + log.info("Recommended re-scaling for {} stage to process work units: {}", stage, procWUsWorkerScaling); return Arrays.asList(procWUsWorkerScaling); } protected abstract int calcDerivationSetPoint(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, JobState jobState); - protected ProfileDerivation calcProfileDerivation(String basisProfileName, WorkUnitsSizeSummary remainingWork, String sourceClass, JobState jobState) { - // TODO: implement right-sizing!!! (for now just return unchanged) - return new ProfileDerivation(basisProfileName, ProfileOverlay.unchanged()); + protected ProfileDerivation calcProfileDerivation(String basisProfileName, WorkUnitsSizeSummary remainingWork, String sourceClass, JobState jobState, WorkflowStage stage) { + // Create overlay with stage-specific memory and worker class + ProfileOverlay overlay = createStageSpecificOverlay(jobState, stage); + return new ProfileDerivation(basisProfileName, overlay); } - protected String calcProfileDerivationName(JobState jobState) { + protected String calcProfileDerivationName(JobState jobState, WorkflowStage stage) { // TODO: if we ever return > 1 directive, append a monotonically increasing number to avoid collisions - return DEFAULT_PROFILE_DERIVATION_NAME; + return stage.getProcessingProfileName(); // e.g., "workExecution-proc" } - protected String calcBasisProfileName(JobState jobState) { - return WorkforceProfiles.BASELINE_NAME; // always build upon baseline + protected String calcBasisProfileName(JobState jobState, WorkflowStage stage) { + // Always derive from the global baseline + return WorkforceProfiles.BASELINE_NAME; + } + + /** + * Creates a ProfileOverlay with stage-specific memory and worker class configuration. + * This allows deriving stage-specific profiles from the global baseline. + */ + private ProfileOverlay createStageSpecificOverlay(JobState jobState, WorkflowStage stage) { + List overlayPairs = new java.util.ArrayList<>(); + + // Add stage-specific memory if configured + String memoryKey = getStageMemoryConfigKey(stage); + if (jobState.contains(memoryKey)) { + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + jobState.getProp(memoryKey) + )); + } + + // Add stage-specific worker class + String workerClass = getWorkerClassForStage(stage); + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + workerClass + )); + + return overlayPairs.isEmpty() ? ProfileOverlay.unchanged() : new ProfileOverlay.Adding(overlayPairs); + } + + /** + * Returns the configuration key for stage-specific memory. + */ + private String getStageMemoryConfigKey(WorkflowStage stage) { + switch (stage) { + case WORK_DISCOVERY: + case COMMIT: + return GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB; + case WORK_EXECUTION: + return GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB; + default: + throw new IllegalArgumentException("Unknown stage: " + stage); + } + } + + /** + * Gets the worker class for a specific workflow stage. + */ + private String getWorkerClassForStage(WorkflowStage stage) { + switch (stage) { + case WORK_DISCOVERY: + return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; + case WORK_EXECUTION: + return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; + case COMMIT: + return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; + default: + return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; + } } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java new file mode 100644 index 00000000000..65b4240695d --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java @@ -0,0 +1,100 @@ +/* + * 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.temporal.ddm.worker; + +import java.util.concurrent.TimeUnit; + +import com.typesafe.config.Config; + +import io.temporal.client.WorkflowClient; +import io.temporal.worker.WorkerOptions; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.cluster.AbstractTemporalWorker; +import org.apache.gobblin.temporal.ddm.activity.impl.CommitActivityImpl; +import org.apache.gobblin.temporal.ddm.activity.impl.DeleteWorkDirsActivityImpl; +import org.apache.gobblin.temporal.ddm.activity.impl.GenerateWorkUnitsImpl; +import org.apache.gobblin.temporal.ddm.activity.impl.RecommendScalingForWorkUnitsLinearHeuristicImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.CommitStepWorkflowImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.ExecuteGobblinWorkflowImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.GenerateWorkUnitsWorkflowImpl; +import org.apache.gobblin.temporal.workflows.metrics.SubmitGTEActivityImpl; +import org.apache.gobblin.util.ConfigUtils; + + +/** + * Specialized worker for Work Discovery and Commit stages. + * This worker registers activities for: + * - GenerateWorkUnits (Work Discovery) + * - RecommendScaling (lightweight scaling recommendation) + * - CommitActivity (Commit) + * - DeleteWorkDirs (Cleanup) + * + * Runs on containers with stage-specific memory for lightweight operations. + * Polls the discovery/commit task queue to ensure activities run on appropriately-sized containers. + */ +public class DiscoveryCommitWorker extends AbstractTemporalWorker { + public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; + public int maxExecutionConcurrency; + + public DiscoveryCommitWorker(Config config, WorkflowClient workflowClient) { + super(config, workflowClient); + this.maxExecutionConcurrency = ConfigUtils.getInt(config, GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER); + } + + @Override + protected Class[] getWorkflowImplClasses() { + return new Class[] { + ExecuteGobblinWorkflowImpl.class, + GenerateWorkUnitsWorkflowImpl.class, + CommitStepWorkflowImpl.class + }; + } + + @Override + protected Object[] getActivityImplInstances() { + // Register activities for both Discovery and Commit stages + return new Object[] { + new SubmitGTEActivityImpl(), + new GenerateWorkUnitsImpl(), // Work Discovery + new RecommendScalingForWorkUnitsLinearHeuristicImpl(), // Scaling recommendation + new CommitActivityImpl(), // Commit + new DeleteWorkDirsActivityImpl() // Cleanup + }; + } + + @Override + protected WorkerOptions createWorkerOptions() { + return WorkerOptions.newBuilder() + .setDefaultDeadlockDetectionTimeout(TimeUnit.SECONDS.toMillis(DEADLOCK_DETECTION_TIMEOUT_SECONDS)) + .setMaxConcurrentActivityExecutionSize(this.maxExecutionConcurrency) + .setMaxConcurrentLocalActivityExecutionSize(this.maxExecutionConcurrency) + .setMaxConcurrentWorkflowTaskExecutionSize(this.maxExecutionConcurrency) + .build(); + } + + @Override + protected String getTaskQueue() { + return ConfigUtils.getString( + config, + GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE + ); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java new file mode 100644 index 00000000000..e69de29bb2d diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java new file mode 100644 index 00000000000..fb7dc41718b --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java @@ -0,0 +1,87 @@ +/* + * 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.temporal.ddm.worker; + +import java.util.concurrent.TimeUnit; + +import com.typesafe.config.Config; + +import io.temporal.client.WorkflowClient; +import io.temporal.worker.WorkerOptions; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.cluster.AbstractTemporalWorker; +import org.apache.gobblin.temporal.ddm.activity.impl.ProcessWorkUnitImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.NestingExecOfProcessWorkUnitWorkflowImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.ProcessWorkUnitsWorkflowImpl; +import org.apache.gobblin.util.ConfigUtils; + + +/** + * Specialized worker for Work Execution stage. + * This worker only registers activities for: + * - ProcessWorkUnit (Work Execution) + * + * Runs on containers with stage-specific memory for work execution operations. + * Polls the execution task queue to ensure activities run on appropriately-sized containers. + */ +public class ExecutionWorker extends AbstractTemporalWorker { + public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; + public int maxExecutionConcurrency; + + public ExecutionWorker(Config config, WorkflowClient workflowClient) { + super(config, workflowClient); + this.maxExecutionConcurrency = ConfigUtils.getInt(config, GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER); + } + + @Override + protected Class[] getWorkflowImplClasses() { + return new Class[] { + ProcessWorkUnitsWorkflowImpl.class, + NestingExecOfProcessWorkUnitWorkflowImpl.class + }; + } + + @Override + protected Object[] getActivityImplInstances() { + // Only register activities for Execution stage + return new Object[] { + new ProcessWorkUnitImpl() // Work Execution only + }; + } + + @Override + protected WorkerOptions createWorkerOptions() { + return WorkerOptions.newBuilder() + .setDefaultDeadlockDetectionTimeout(TimeUnit.SECONDS.toMillis(DEADLOCK_DETECTION_TIMEOUT_SECONDS)) + .setMaxConcurrentActivityExecutionSize(this.maxExecutionConcurrency) + .setMaxConcurrentLocalActivityExecutionSize(this.maxExecutionConcurrency) + .setMaxConcurrentWorkflowTaskExecutionSize(this.maxExecutionConcurrency) + .build(); + } + + @Override + protected String getTaskQueue() { + return ConfigUtils.getString( + config, + GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE + ); + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java index 12fe6c4d849..cdc394e3e50 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java @@ -17,15 +17,19 @@ package org.apache.gobblin.temporal.ddm.worker; +import java.util.Arrays; import java.util.concurrent.TimeUnit; import com.typesafe.config.Config; import io.temporal.client.WorkflowClient; +import io.temporal.worker.Worker; +import io.temporal.worker.WorkerFactory; import io.temporal.worker.WorkerOptions; import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.cluster.AbstractTemporalWorker; +import org.apache.gobblin.temporal.cluster.WorkerConfig; import org.apache.gobblin.temporal.ddm.activity.impl.CommitActivityImpl; import org.apache.gobblin.temporal.ddm.activity.impl.DeleteWorkDirsActivityImpl; import org.apache.gobblin.temporal.ddm.activity.impl.GenerateWorkUnitsImpl; @@ -40,7 +44,19 @@ import org.apache.gobblin.util.ConfigUtils; -/** Worker for the {@link ProcessWorkUnitsWorkflowImpl} super-workflow */ +/** + * Unified worker that handles ALL workflow stages (Discovery, Execution, Commit). + * + * This worker polls the DEFAULT task queue and is used when: + * - Dynamic scaling is disabled + * - Small jobs where scaling doesn't make sense + * + * When dynamic scaling is enabled, activities are routed to stage-specific queues + * and handled by specialized workers (DiscoveryCommitWorker, ExecutionWorker). + * + * This prevents resource mismatches (e.g., execution tasks requiring 64GB running + * on a baseline container with only 8GB). + */ public class WorkFulfillmentWorker extends AbstractTemporalWorker { public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; // TODO: make configurable! public int maxExecutionConcurrency; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java new file mode 100644 index 00000000000..bd69de920a4 --- /dev/null +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java @@ -0,0 +1,111 @@ +/* + * 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.temporal.ddm.workflow; + +import lombok.Getter; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; + +/** + * Represents the different stages of a Gobblin Temporal workflow. + * Each stage has independent resource configuration (memory, OOM limits) and dedicated containers. + * + *

Stages: + *

    + *
  • WORK_DISCOVERY: Discovers data sources, generates work units, and handles commits (1 container, lightweight operations)
  • + *
  • WORK_EXECUTION: Processes work units to transform and load data (20+ containers)
  • + *
+ * + *

Each stage has: + *

    + *
  • Dedicated task queue for activity routing
  • + *
  • Specialized worker class
  • + *
  • Independent memory configuration
  • + *
  • Stage-specific OOM handling
  • + *
+ */ +@Getter +public enum WorkflowStage { + WORK_DISCOVERY("workDiscovery", GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE), + WORK_EXECUTION("workExecution", GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE), + COMMIT("commit", GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE); + + private final String profileBaseName; + private final String taskQueueConfigKey; + private final String defaultTaskQueue; + + WorkflowStage(String profileBaseName, String taskQueueConfigKey, String defaultTaskQueue) { + this.profileBaseName = profileBaseName; + this.taskQueueConfigKey = taskQueueConfigKey; + this.defaultTaskQueue = defaultTaskQueue; + } + + /** + * Returns the baseline profile name pattern for this stage. + * Used for naming derived profiles and identifying stage from profile names. + * Note: Stage-specific profiles are derived dynamically from the global baseline, + * not created as separate baseline profiles. + * Example: "baseline-workDiscovery", "baseline-workExecution" + */ + public String getBaselineProfileName() { + return "baseline-" + profileBaseName; + } + + /** + * Returns the processing profile name for this stage. + * Example: "workDiscovery-proc", "workExecution-proc" + */ + public String getProcessingProfileName() { + return profileBaseName + "-proc"; + } + + /** + * Returns the task queue for this stage, reading from config or using default. + * Example: "GobblinTemporalDiscoveryCommitQueue", "GobblinTemporalExecutionQueue" + * + * @param config the configuration to read from + * @return the task queue name for this stage + */ + public String getTaskQueue(com.typesafe.config.Config config) { + return config.hasPath(taskQueueConfigKey) + ? config.getString(taskQueueConfigKey) + : defaultTaskQueue; + } + + /** + * Determines the workflow stage from a profile name. + * Used by OOM handler to identify which stage a container belongs to. + * + * @param profileName the profile name (e.g., "workExecution-proc", "baseline-workDiscovery") + * @return the corresponding WorkflowStage + */ + public static WorkflowStage fromProfileName(String profileName) { + if (profileName == null || profileName.isEmpty()) { + return WORK_DISCOVERY; // Global baseline used by initial container + } + for (WorkflowStage stage : values()) { + if (profileName.contains(stage.getProfileBaseName())) { + return stage; + } + } + return WORK_EXECUTION; // Default to execution if no match + } +} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java index 50d587c4efe..8f8819c7d71 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java @@ -29,6 +29,7 @@ import org.apache.gobblin.temporal.ddm.work.CommitStats; import org.apache.gobblin.temporal.ddm.work.WUProcessingSpec; import org.apache.gobblin.temporal.ddm.workflow.CommitStepWorkflow; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; @Slf4j @@ -36,7 +37,11 @@ public class CommitStepWorkflowImpl implements CommitStepWorkflow { @Override public CommitStats commit(WUProcessingSpec workSpec, final Properties props) { - final CommitActivity activityStub = Workflow.newActivityStub(CommitActivity.class, ActivityType.COMMIT.buildActivityOptions(props, true)); + // Route Commit to commit queue + com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(props); + String commitTaskQueue = WorkflowStage.COMMIT.getTaskQueue(config); + final CommitActivity activityStub = Workflow.newActivityStub(CommitActivity.class, + ActivityType.COMMIT.buildActivityOptions(props, true, commitTaskQueue)); CommitStats commitGobblinStats = activityStub.commit(workSpec); if (commitGobblinStats.getOptFailure().isPresent()) { throw ApplicationFailure.newNonRetryableFailureWithCause( diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 05fa2a8546b..407a1dbfd1d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -52,6 +52,7 @@ import org.apache.gobblin.temporal.ddm.activity.GenerateWorkUnits; import org.apache.gobblin.temporal.ddm.activity.RecommendScalingForWorkUnits; import org.apache.gobblin.temporal.ddm.launcher.ProcessWorkUnitsJobLauncher; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.ddm.util.JobStateUtils; import org.apache.gobblin.temporal.ddm.util.TemporalWorkFlowUtils; import org.apache.gobblin.temporal.ddm.work.CommitStats; @@ -95,8 +96,11 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event WUProcessingSpec wuSpec = createProcessingSpec(jobProps, eventSubmitterContext); boolean isSuccessful = false; try (Closer closer = Closer.create()) { + // Route GenerateWorkUnits to discovery queue + Config config = ConfigFactory.parseProperties(temporalJobProps); + String discoveryTaskQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); final GenerateWorkUnits genWUsActivityStub = Workflow.newActivityStub(GenerateWorkUnits.class, - ActivityType.GENERATE_WORKUNITS.buildActivityOptions(temporalJobProps, true)); + ActivityType.GENERATE_WORKUNITS.buildActivityOptions(temporalJobProps, true, discoveryTaskQueue)); GenerateWorkUnitsResult generateWorkUnitResult = genWUsActivityStub.generateWorkUnits(jobProps, eventSubmitterContext); optGenerateWorkUnitResult = Optional.of(generateWorkUnitResult); WorkUnitsSizeSummary wuSizeSummary = generateWorkUnitResult.getWorkUnitsSizeSummary(); @@ -107,11 +111,13 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event CommitStats commitStats = CommitStats.createEmpty(); if (numWUsGenerated > 0) { TimeBudget timeBudget = calcWUProcTimeBudget(jobSuccessTimer.getStartTime(), wuSizeSummary, jobProps); + // Route RecommendScaling to discovery queue final RecommendScalingForWorkUnits recommendScalingStub = Workflow.newActivityStub(RecommendScalingForWorkUnits.class, - ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false)); + ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false, discoveryTaskQueue)); List scalingDirectives = - recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps); - log.info("Recommended scaling to process WUs within {}: {}", timeBudget, scalingDirectives); + recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps, + WorkflowStage.WORK_EXECUTION); + log.info("Recommended scaling for WORK_EXECUTION stage to process WUs within {}: {}", timeBudget, scalingDirectives); try { ScalingDirectivesRecipient recipient = createScalingDirectivesRecipient(jobProps, closer); List adjustedScalingDirectives = adjustRecommendedScaling(jobProps, scalingDirectives); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java index a5a2247ed2e..87d906c0223 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java @@ -26,6 +26,7 @@ import org.apache.gobblin.temporal.ddm.activity.ActivityType; import org.apache.gobblin.temporal.ddm.activity.ProcessWorkUnit; import org.apache.gobblin.temporal.ddm.work.WorkUnitClaimCheck; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.util.nesting.workflow.AbstractNestingExecWorkflowImpl; @@ -34,8 +35,11 @@ public class NestingExecOfProcessWorkUnitWorkflowImpl extends AbstractNestingExe @Override protected Promise launchAsyncActivity(final WorkUnitClaimCheck wu, final Properties props) { + // Route ProcessWorkUnit to execution queue + com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(props); + String executionTaskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); final ProcessWorkUnit processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, - ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true)); + ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true, executionTaskQueue)); return Async.function(processWorkUnitStub::processWorkUnit, wu); } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java index 60aceee0e65..829d0aa8798 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java @@ -105,6 +105,7 @@ private CommitStats performCommitIfAnyWorkUnitsProcessed(WUProcessingSpec workSp log.error("No work units processed, so no commit attempted."); return CommitStats.createEmpty(); } + CommitStepWorkflow commitWorkflow = createCommitStepWorkflow(searchAttributes); CommitStats result = commitWorkflow.commit(workSpec, props); if (result.getNumCommittedWorkUnits() == 0) { @@ -153,4 +154,5 @@ protected CommitStepWorkflow createCommitStepWorkflow(Map search return Workflow.newChildWorkflowStub(CommitStepWorkflow.class, childOpts); } + } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 336d357f237..94b2dd4fa29 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -84,7 +84,13 @@ protected UnrecognizedProfile(ScalingDirective directive) { /** create new plan with the initial, baseline worker profile using `baselineConfig` at `initialSetPoint` */ public WorkforcePlan(Config baselineConfig, int initialSetPoint) { this.profiles = WorkforceProfiles.withBaseline(baselineConfig); - this.staffing = WorkforceStaffing.initialize(initialSetPoint); + this.staffing = WorkforceStaffing.initialize(0); + // Initial containers use the global baseline profile + this.staffing.reviseStaffing( + WorkforceProfiles.BASELINE_NAME, + initialSetPoint, + 0 + ); this.lastRevisionEpochMillis = 0; } @@ -174,4 +180,5 @@ WorkerProfile peepProfile(String profileName) throws WorkforceProfiles.UnknownPr WorkerProfile peepBaselineProfile() throws WorkforceProfiles.UnknownProfileException { return profiles.getOrThrow(WorkforceProfiles.BASELINE_NAME); } + } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 0010a45ff8e..d72f39da0ca 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -27,6 +27,7 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.util.ConfigUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -38,6 +39,9 @@ import lombok.extern.slf4j.Slf4j; +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.dynamic.ProfileDerivation; import org.apache.gobblin.temporal.dynamic.ProfileOverlay; import org.apache.gobblin.temporal.dynamic.ScalingDirective; @@ -65,25 +69,142 @@ public class DynamicScalingYarnService extends YarnService { private final WorkforceStaffing actualWorkforceStaffing; /** this holds the current total workforce plan as per latest received scaling directives */ private final WorkforcePlan workforcePlan; + private final int initialContainers; protected final Queue removedContainerIds; private final AtomicLong profileNameSuffixGenerator; + private final boolean dynamicScalingEnabled; public DynamicScalingYarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { super(config, applicationName, applicationId, yarnConfiguration, fs, eventBus); + this.dynamicScalingEnabled = ConfigUtils.getBoolean(config, + GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, false); + this.initialContainers = this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); + this.actualWorkforceStaffing = WorkforceStaffing.initialize(0); - this.workforcePlan = new WorkforcePlan(this.config, this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY)); + // Initialize workforce plan: + // - For dynamic scaling: start with 0 baseline, then add stage-specific profiles + // - For traditional mode: initialize baseline with configured initial containers + int baselineSetPoint = this.dynamicScalingEnabled ? 0 : this.initialContainers; + this.workforcePlan = new WorkforcePlan(this.config, baselineSetPoint); this.removedContainerIds = new ConcurrentLinkedQueue<>(); this.profileNameSuffixGenerator = new AtomicLong(); + + // For dynamic scaling, add stage-specific profiles derived from baseline + if (this.dynamicScalingEnabled) { + initializeDynamicScalingProfiles(); + } + } + + /** + * Initializes stage-specific worker profiles for dynamic scaling mode. + * Creates two profiles derived from baseline: + * 1. DiscoveryCommitWorker - for discovery/commit activities + * 2. ExecutionWorker - for execution activities + */ + private void initializeDynamicScalingProfiles() { + log.info("Initializing stage-specific profiles"); + long currTimeMillis = System.currentTimeMillis(); + List initialDirectives = new ArrayList<>(); + + // Container 1: DiscoveryCommitWorker (for discovery/commit activities) + ProfileOverlay discoveryCommitOverlay = createDiscoveryCommitProfileOverlay(); + initialDirectives.add(new ScalingDirective( + "initial-discovery-commit", + 1, // setPoint = 1 container + currTimeMillis, + WorkforceProfiles.BASELINE_NAME, + discoveryCommitOverlay + )); + + // Container 2: ExecutionWorker (for execution activities) + ProfileOverlay executionOverlay = createExecutionProfileOverlay(); + initialDirectives.add(new ScalingDirective( + "initial-execution", + 1, // setPoint = 1 container + currTimeMillis + EPSILON_MIILIS, + WorkforceProfiles.BASELINE_NAME, + executionOverlay + )); + + // Apply initial directives to workforce plan + this.workforcePlan.reviseWhenNewer(initialDirectives, ire -> { + log.error("Failed to create stage-specific profiles", ire); + throw new RuntimeException("Failed to initialize stage-specific profiles for dynamic scaling", ire); + }); + + log.info("Initialized {} stage-specific profiles for dynamic scaling", initialDirectives.size()); } @Override protected synchronized void requestInitialContainers() { + log.info("Requesting initial containers based on workforce plan"); + // Calculate deltas between plan and current staffing, then request containers StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); requestNewContainersForStaffingDeltas(deltas); } + private ProfileOverlay createDiscoveryCommitProfileOverlay() { + List overlayPairs = new ArrayList<>(); + + // Set worker class + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker" + )); + + // Set Helix tag + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "discovery-commit" + )); + + // Set stage-specific memory (discovery/commit operations are typically lightweight) + // Falls back to global CONTAINER_MEMORY_MBS_KEY if stage-specific memory not configured + if (this.config.hasPath(GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB)) { + String discoveryCommitMemoryMb = this.config.getString( + GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB); + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + discoveryCommitMemoryMb + )); + log.info("Setting discovery-commit container memory to {} MB", discoveryCommitMemoryMb); + } + + return new ProfileOverlay.Adding(overlayPairs); + } + + private ProfileOverlay createExecutionProfileOverlay() { + List overlayPairs = new ArrayList<>(); + + // Set worker class + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker" + )); + + // Set Helix tag + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "execution" + )); + + // Set stage-specific memory (execution operations are typically memory-intensive) + // Falls back to global CONTAINER_MEMORY_MBS_KEY if stage-specific memory not configured + if (this.config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB)) { + String executionMemoryMb = this.config.getString( + GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB); + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + executionMemoryMb + )); + log.info("Setting execution container memory to {} MB", executionMemoryMb); + } + + return new ProfileOverlay.Adding(overlayPairs); + } + /** * Handle the completion of a container. A new container will be requested to replace the one * that just exited depending on the exit status. @@ -226,12 +347,16 @@ private void handleAbortedContainer(ContainerId completedContainerId, ContainerI } private synchronized void handleContainerExitedWithOOM(ContainerId completedContainerId, ContainerInfo completedContainerInfo) { - log.info("Container {} for profile {} exited with OOM, starting to launch a replacement container", - completedContainerId, completedContainerInfo.getWorkerProfileName()); + WorkerProfile workerProfile = completedContainerInfo.getWorkerProfile(); + + // Determine which workflow stage this container belongs to + WorkflowStage stage = WorkflowStage.fromProfileName(workerProfile.getName()); + + log.info("Container {} for profile {} (stage: {}) exited with OOM, starting to launch a replacement container", + completedContainerId, completedContainerInfo.getWorkerProfileName(), stage); List scalingDirectives = new ArrayList<>(); - WorkerProfile workerProfile = completedContainerInfo.getWorkerProfile(); long currTimeMillis = System.currentTimeMillis(); // Update the current staffing to reflect the container that exited with OOM int currNumContainers = this.actualWorkforceStaffing.getStaffing(workerProfile.getName()).orElse(0); @@ -242,20 +367,29 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont scalingDirectives.add(new ScalingDirective(workerProfile.getName(), currNumContainers - 1, currTimeMillis)); } - // Request a replacement container + // Get stage-specific OOM configuration + int memoryMultiplier = getMemoryMultiplierForStage(stage); + int maxMemoryMbs = getMaxMemoryForStage(stage); + + // Request a replacement container with stage-specific limits int currContainerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - if (currContainerMemoryMbs >= MAX_REPLACEMENT_CONTAINER_MEMORY_MBS) { - log.warn("Container {} already had max allowed memory {} MBs. Not requesting a replacement container.", - completedContainerId, currContainerMemoryMbs); + if (currContainerMemoryMbs >= maxMemoryMbs) { + log.warn("Container {} for stage {} already had max allowed memory {} MBs (stage max: {} MBs). Not requesting a replacement container.", + completedContainerId, stage, currContainerMemoryMbs, maxMemoryMbs); return; } - int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER, - MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); - Optional optProfileDerivation = Optional.of(new ProfileDerivation(workerProfile.getName(), - new ProfileOverlay.Adding(new ProfileOverlay.KVPair(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, newContainerMemoryMbs + "")) - )); + int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * memoryMultiplier, maxMemoryMbs); + + log.info("Creating OOM replacement for stage {} with memory: {} MB -> {} MB (max: {} MB)", + stage, currContainerMemoryMbs, newContainerMemoryMbs, maxMemoryMbs); + + // Derive from global baseline with stage-specific overlays + ProfileOverlay overlay = createStageSpecificOOMOverlay(stage, newContainerMemoryMbs); + Optional optProfileDerivation = Optional.of( + new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, overlay) + ); scalingDirectives.add(new ScalingDirective( - DEFAULT_REPLACEMENT_CONTAINER_WORKER_PROFILE_NAME_PREFIX + "-" + profileNameSuffixGenerator.getAndIncrement(), + stage.getProfileBaseName() + "-oomReplacement-" + profileNameSuffixGenerator.getAndIncrement(), 1, currTimeMillis + EPSILON_MIILIS, // Each scaling directive should have a newer timestamp than the previous one optProfileDerivation @@ -263,4 +397,91 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); } + /** + * Gets the memory multiplier for OOM retries for a specific workflow stage. + */ + private int getMemoryMultiplierForStage(WorkflowStage stage) { + String key; + int defaultValue; + + switch (stage) { + case WORK_DISCOVERY: + case COMMIT: + key = GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER; + defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER; + break; + case WORK_EXECUTION: + key = GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; + defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; + break; + default: + return DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER; + } + + return config.hasPath(key) ? config.getInt(key) : defaultValue; + } + + /** + * Gets the maximum memory allowed for OOM retries for a specific workflow stage. + */ + private int getMaxMemoryForStage(WorkflowStage stage) { + String key; + int defaultValue; + + switch (stage) { + case WORK_DISCOVERY: + case COMMIT: + key = GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB; + defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB; + break; + case WORK_EXECUTION: + key = GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB; + defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB; + break; + default: + return MAX_REPLACEMENT_CONTAINER_MEMORY_MBS; + } + + return config.hasPath(key) ? config.getInt(key) : defaultValue; + } + + /** + * Creates a ProfileOverlay for OOM replacement with stage-specific memory and worker class. + * This derives from the global baseline, ensuring task queue routing is preserved. + */ + private ProfileOverlay createStageSpecificOOMOverlay(WorkflowStage stage, int newMemoryMbs) { + List overlayPairs = new ArrayList<>(); + + // Add increased memory + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + String.valueOf(newMemoryMbs) + )); + + // Add stage-specific worker class to ensure correct task queue routing + String workerClass = getWorkerClassForStage(stage); + overlayPairs.add(new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + workerClass + )); + + return new ProfileOverlay.Adding(overlayPairs); + } + + /** + * Gets the worker class for a specific workflow stage. + */ + private String getWorkerClassForStage(WorkflowStage stage) { + switch (stage) { + case WORK_DISCOVERY: + return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; + case WORK_EXECUTION: + return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; + case COMMIT: + return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; + default: + return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; + } + } + } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index 2818982baba..df5976bf271 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -104,6 +104,7 @@ import org.apache.gobblin.yarn.GobblinYarnEventConstants; import org.apache.gobblin.yarn.GobblinYarnMetricTagNames; import org.apache.gobblin.yarn.YarnHelixUtils; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.dynamic.WorkerProfile; import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; @@ -514,6 +515,11 @@ private String buildContainerCommand(Container container, String workerProfileNa Optional.of(workerProfileConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : Optional.absent(); + // Extract worker class from profile config to pass as system property + String workerClass = ConfigUtils.getString(workerProfileConfig, + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS); + String containerProcessName = GobblinTemporalYarnTaskRunner.class.getSimpleName(); StringBuilder containerCommand = new StringBuilder() .append(ApplicationConstants.Environment.JAVA_HOME.$()).append("/bin/java") @@ -522,6 +528,7 @@ private String buildContainerCommand(Container container, String workerProfileNa .append(" -D").append(GobblinYarnConfigurationKeys.JVM_USER_TIMEZONE_CONFIG).append("=").append(this.containerTimezone) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT) + .append(" -D").append(GobblinTemporalConfigurationKeys.WORKER_CLASS).append("=").append(workerClass) .append(" ").append(JvmUtils.formatJvmArguments(workerJvmArgs)) .append(" ").append(this.proxyJvmArgs) .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName()) From 383a7cd362c637b416bdaca1a63f7d0a5e450146 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Thu, 11 Dec 2025 16:03:24 +0530 Subject: [PATCH 02/13] route non execute activities to default queue --- .../GobblinTemporalConfigurationKeys.java | 19 +-- ...tractRecommendScalingForWorkUnitsImpl.java | 27 ++-- ...calingForWorkUnitsLinearHeuristicImpl.java | 4 +- .../ddm/worker/DiscoveryCommitWorker.java | 100 --------------- .../temporal/ddm/worker/DiscoveryWorker.java | 0 .../ddm/worker/WorkFulfillmentWorker.java | 12 +- .../temporal/ddm/workflow/WorkflowStage.java | 46 ++++--- .../workflow/impl/CommitStepWorkflowImpl.java | 6 +- .../impl/ExecuteGobblinWorkflowImpl.java | 10 +- ...tingExecOfProcessWorkUnitWorkflowImpl.java | 20 ++- .../yarn/DynamicScalingYarnService.java | 121 ++++-------------- 11 files changed, 98 insertions(+), 267 deletions(-) delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java delete mode 100644 gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index 0326b317b6d..6411862e3fb 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -38,10 +38,7 @@ public interface GobblinTemporalConfigurationKeys { String GOBBLIN_TEMPORAL_TASK_QUEUE = PREFIX + "task.queue.name"; String DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue"; - // Stage-specific task queues for worker specialization - // Discovery and Commit share the same queue (lightweight operations) - String DISCOVERY_COMMIT_TASK_QUEUE = PREFIX + "discovery.commit.task.queue.name"; - String DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE = "GobblinTemporalDiscoveryCommitQueue"; + // Execution task queue for worker specialization (when dynamic scaling is enabled) String EXECUTION_TASK_QUEUE = PREFIX + "execution.task.queue.name"; String DEFAULT_EXECUTION_TASK_QUEUE = "GobblinTemporalExecutionQueue"; String GOBBLIN_TEMPORAL_JOB_LAUNCHER_PREFIX = PREFIX + "job.launcher."; @@ -138,20 +135,12 @@ public interface GobblinTemporalConfigurationKeys { int DEFAULT_TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = 4; /** - * Stage-specific resource configuration - * Allows independent memory and OOM configuration per workflow stage + * Stage-specific resource configuration for execution workers + * Allows independent memory and OOM configuration for work execution stage */ String STAGE_SPECIFIC_PREFIX = PREFIX + "stage."; - // Discovery/Commit stage configuration (shared by DiscoveryCommitWorker) - // Handles both work discovery and commit activities in the same container - String DISCOVERY_COMMIT_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "discoveryCommit.memory.mb"; - String DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER = STAGE_SPECIFIC_PREFIX + "discoveryCommit.oom.memory.multiplier"; - int DEFAULT_DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER = 2; - String DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "discoveryCommit.oom.max.memory.mb"; - int DEFAULT_DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB = 32768; // 32GB (lightweight operations) - - // Work Execution stage configuration + // Work Execution stage configuration (for ExecutionWorker when dynamic scaling is enabled) String WORK_EXECUTION_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.memory.mb"; String WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = STAGE_SPECIFIC_PREFIX + "workExecution.oom.memory.multiplier"; int DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = 2; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java index dd11f73cf39..7d2532041f0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java @@ -108,32 +108,25 @@ private ProfileOverlay createStageSpecificOverlay(JobState jobState, WorkflowSta /** * Returns the configuration key for stage-specific memory. + * Only WORK_EXECUTION has stage-specific memory config; others use default. */ private String getStageMemoryConfigKey(WorkflowStage stage) { - switch (stage) { - case WORK_DISCOVERY: - case COMMIT: - return GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB; - case WORK_EXECUTION: - return GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB; - default: - throw new IllegalArgumentException("Unknown stage: " + stage); + if (stage == WorkflowStage.WORK_EXECUTION) { + return GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB; } + // Non-execution stages (discovery, commit) use default/baseline memory + return null; } /** * Gets the worker class for a specific workflow stage. + * Returns ExecutionWorker for WORK_EXECUTION, WorkFulfillmentWorker for all others. */ private String getWorkerClassForStage(WorkflowStage stage) { - switch (stage) { - case WORK_DISCOVERY: - return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; - case WORK_EXECUTION: - return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; - case COMMIT: - return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; - default: - return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; + if (stage == WorkflowStage.WORK_EXECUTION) { + return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; } + // All non-execution stages use WorkFulfillmentWorker (discovery, commit, etc.) + return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java index 777f7e9c200..55b05484488 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java @@ -97,7 +97,9 @@ protected int calcDerivationSetPoint(WorkUnitsSizeSummary remainingWork, String int recommendedNumContainers = Math.max(numContainerForWUs, numContainerForThroughout); log.info("Recommended auto-scaling: {} containers, no. of containers considering throughput: {}, no. of containers considering WUs: {}", recommendedNumContainers, numContainerForThroughout, numContainerForWUs); - +/** + * + */ return recommendedNumContainers; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java deleted file mode 100644 index 65b4240695d..00000000000 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryCommitWorker.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.temporal.ddm.worker; - -import java.util.concurrent.TimeUnit; - -import com.typesafe.config.Config; - -import io.temporal.client.WorkflowClient; -import io.temporal.worker.WorkerOptions; - -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.cluster.AbstractTemporalWorker; -import org.apache.gobblin.temporal.ddm.activity.impl.CommitActivityImpl; -import org.apache.gobblin.temporal.ddm.activity.impl.DeleteWorkDirsActivityImpl; -import org.apache.gobblin.temporal.ddm.activity.impl.GenerateWorkUnitsImpl; -import org.apache.gobblin.temporal.ddm.activity.impl.RecommendScalingForWorkUnitsLinearHeuristicImpl; -import org.apache.gobblin.temporal.ddm.workflow.impl.CommitStepWorkflowImpl; -import org.apache.gobblin.temporal.ddm.workflow.impl.ExecuteGobblinWorkflowImpl; -import org.apache.gobblin.temporal.ddm.workflow.impl.GenerateWorkUnitsWorkflowImpl; -import org.apache.gobblin.temporal.workflows.metrics.SubmitGTEActivityImpl; -import org.apache.gobblin.util.ConfigUtils; - - -/** - * Specialized worker for Work Discovery and Commit stages. - * This worker registers activities for: - * - GenerateWorkUnits (Work Discovery) - * - RecommendScaling (lightweight scaling recommendation) - * - CommitActivity (Commit) - * - DeleteWorkDirs (Cleanup) - * - * Runs on containers with stage-specific memory for lightweight operations. - * Polls the discovery/commit task queue to ensure activities run on appropriately-sized containers. - */ -public class DiscoveryCommitWorker extends AbstractTemporalWorker { - public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; - public int maxExecutionConcurrency; - - public DiscoveryCommitWorker(Config config, WorkflowClient workflowClient) { - super(config, workflowClient); - this.maxExecutionConcurrency = ConfigUtils.getInt(config, GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, - GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER); - } - - @Override - protected Class[] getWorkflowImplClasses() { - return new Class[] { - ExecuteGobblinWorkflowImpl.class, - GenerateWorkUnitsWorkflowImpl.class, - CommitStepWorkflowImpl.class - }; - } - - @Override - protected Object[] getActivityImplInstances() { - // Register activities for both Discovery and Commit stages - return new Object[] { - new SubmitGTEActivityImpl(), - new GenerateWorkUnitsImpl(), // Work Discovery - new RecommendScalingForWorkUnitsLinearHeuristicImpl(), // Scaling recommendation - new CommitActivityImpl(), // Commit - new DeleteWorkDirsActivityImpl() // Cleanup - }; - } - - @Override - protected WorkerOptions createWorkerOptions() { - return WorkerOptions.newBuilder() - .setDefaultDeadlockDetectionTimeout(TimeUnit.SECONDS.toMillis(DEADLOCK_DETECTION_TIMEOUT_SECONDS)) - .setMaxConcurrentActivityExecutionSize(this.maxExecutionConcurrency) - .setMaxConcurrentLocalActivityExecutionSize(this.maxExecutionConcurrency) - .setMaxConcurrentWorkflowTaskExecutionSize(this.maxExecutionConcurrency) - .build(); - } - - @Override - protected String getTaskQueue() { - return ConfigUtils.getString( - config, - GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, - GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE - ); - } -} diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/DiscoveryWorker.java deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java index cdc394e3e50..9d860308af3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java @@ -45,14 +45,14 @@ /** - * Unified worker that handles ALL workflow stages (Discovery, Execution, Commit). + * Unified worker that handles ALL workflow stages except execution (Discovery, Commit, etc.). * - * This worker polls the DEFAULT task queue and is used when: - * - Dynamic scaling is disabled - * - Small jobs where scaling doesn't make sense + * This worker polls the DEFAULT task queue and is used: + * - Always for discovery, commit, and other non-execution activities + * - For execution activities when dynamic scaling is disabled * - * When dynamic scaling is enabled, activities are routed to stage-specific queues - * and handled by specialized workers (DiscoveryCommitWorker, ExecutionWorker). + * When dynamic scaling is enabled, execution activities are routed to the execution queue + * and handled by the specialized ExecutionWorker. * * This prevents resource mismatches (e.g., execution tasks requiring 64GB running * on a baseline container with only 8GB). diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java index bd69de920a4..82125f51ffd 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java @@ -23,30 +23,28 @@ /** * Represents the different stages of a Gobblin Temporal workflow. - * Each stage has independent resource configuration (memory, OOM limits) and dedicated containers. - * + * *

Stages: *

    - *
  • WORK_DISCOVERY: Discovers data sources, generates work units, and handles commits (1 container, lightweight operations)
  • - *
  • WORK_EXECUTION: Processes work units to transform and load data (20+ containers)
  • + *
  • WORK_DISCOVERY: Discovers data sources, generates work units (uses default queue)
  • + *
  • WORK_EXECUTION: Processes work units to transform and load data (uses execution queue when dynamic scaling enabled)
  • + *
  • COMMIT: Commits work units (uses default queue)
  • *
* - *

Each stage has: + *

Queue routing: *

    - *
  • Dedicated task queue for activity routing
  • - *
  • Specialized worker class
  • - *
  • Independent memory configuration
  • - *
  • Stage-specific OOM handling
  • + *
  • Dynamic scaling OFF: All stages use default queue
  • + *
  • Dynamic scaling ON: WORK_EXECUTION uses dedicated execution queue, others use default queue
  • *
*/ @Getter public enum WorkflowStage { - WORK_DISCOVERY("workDiscovery", GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, - GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE), + WORK_DISCOVERY("workDiscovery", GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE), WORK_EXECUTION("workExecution", GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE), - COMMIT("commit", GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_TASK_QUEUE, - GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_TASK_QUEUE); + COMMIT("commit", GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE); private final String profileBaseName; private final String taskQueueConfigKey; @@ -94,18 +92,26 @@ public String getTaskQueue(com.typesafe.config.Config config) { * Determines the workflow stage from a profile name. * Used by OOM handler to identify which stage a container belongs to. * - * @param profileName the profile name (e.g., "workExecution-proc", "baseline-workDiscovery") + * In the 2-worker model: + * - Empty/null profile name = baseline/default containers (WorkFulfillmentWorker) → WORK_DISCOVERY + * - Profile containing "workExecution" or "execution" = ExecutionWorker → WORK_EXECUTION + * - All other profiles = default containers → WORK_DISCOVERY + * + * @param profileName the profile name (e.g., "initial-execution", "workExecution-oomReplacement-1", "" for baseline) * @return the corresponding WorkflowStage */ public static WorkflowStage fromProfileName(String profileName) { + // Empty or null = baseline/default containers (WorkFulfillmentWorker) if (profileName == null || profileName.isEmpty()) { - return WORK_DISCOVERY; // Global baseline used by initial container + return WORK_DISCOVERY; // Represents default/non-execution work } - for (WorkflowStage stage : values()) { - if (profileName.contains(stage.getProfileBaseName())) { - return stage; - } + + // Check if this is an execution worker profile + if (profileName.contains("workExecution") || profileName.contains("execution")) { + return WORK_EXECUTION; } - return WORK_EXECUTION; // Default to execution if no match + + // All other profiles default to non-execution work (WorkFulfillmentWorker) + return WORK_DISCOVERY; } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java index 8f8819c7d71..5712dcedd40 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java @@ -29,7 +29,6 @@ import org.apache.gobblin.temporal.ddm.work.CommitStats; import org.apache.gobblin.temporal.ddm.work.WUProcessingSpec; import org.apache.gobblin.temporal.ddm.workflow.CommitStepWorkflow; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; @Slf4j @@ -37,11 +36,8 @@ public class CommitStepWorkflowImpl implements CommitStepWorkflow { @Override public CommitStats commit(WUProcessingSpec workSpec, final Properties props) { - // Route Commit to commit queue - com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(props); - String commitTaskQueue = WorkflowStage.COMMIT.getTaskQueue(config); final CommitActivity activityStub = Workflow.newActivityStub(CommitActivity.class, - ActivityType.COMMIT.buildActivityOptions(props, true, commitTaskQueue)); + ActivityType.COMMIT.buildActivityOptions(props, true)); CommitStats commitGobblinStats = activityStub.commit(workSpec); if (commitGobblinStats.getOptFailure().isPresent()) { throw ApplicationFailure.newNonRetryableFailureWithCause( diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 407a1dbfd1d..008aec54795 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -96,11 +96,9 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event WUProcessingSpec wuSpec = createProcessingSpec(jobProps, eventSubmitterContext); boolean isSuccessful = false; try (Closer closer = Closer.create()) { - // Route GenerateWorkUnits to discovery queue - Config config = ConfigFactory.parseProperties(temporalJobProps); - String discoveryTaskQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); + // GenerateWorkUnits uses default queue (inherits from workflow) final GenerateWorkUnits genWUsActivityStub = Workflow.newActivityStub(GenerateWorkUnits.class, - ActivityType.GENERATE_WORKUNITS.buildActivityOptions(temporalJobProps, true, discoveryTaskQueue)); + ActivityType.GENERATE_WORKUNITS.buildActivityOptions(temporalJobProps, true)); GenerateWorkUnitsResult generateWorkUnitResult = genWUsActivityStub.generateWorkUnits(jobProps, eventSubmitterContext); optGenerateWorkUnitResult = Optional.of(generateWorkUnitResult); WorkUnitsSizeSummary wuSizeSummary = generateWorkUnitResult.getWorkUnitsSizeSummary(); @@ -111,9 +109,9 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event CommitStats commitStats = CommitStats.createEmpty(); if (numWUsGenerated > 0) { TimeBudget timeBudget = calcWUProcTimeBudget(jobSuccessTimer.getStartTime(), wuSizeSummary, jobProps); - // Route RecommendScaling to discovery queue + // RecommendScaling uses default queue (inherits from workflow) final RecommendScalingForWorkUnits recommendScalingStub = Workflow.newActivityStub(RecommendScalingForWorkUnits.class, - ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false, discoveryTaskQueue)); + ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false)); List scalingDirectives = recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps, WorkflowStage.WORK_EXECUTION); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java index 87d906c0223..071389a54e3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java @@ -23,6 +23,7 @@ import io.temporal.workflow.Promise; import io.temporal.workflow.Workflow; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.ddm.activity.ActivityType; import org.apache.gobblin.temporal.ddm.activity.ProcessWorkUnit; import org.apache.gobblin.temporal.ddm.work.WorkUnitClaimCheck; @@ -35,11 +36,22 @@ public class NestingExecOfProcessWorkUnitWorkflowImpl extends AbstractNestingExe @Override protected Promise launchAsyncActivity(final WorkUnitClaimCheck wu, final Properties props) { - // Route ProcessWorkUnit to execution queue + // Route ProcessWorkUnit to execution queue only if dynamic scaling is enabled com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(props); - String executionTaskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); - final ProcessWorkUnit processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, - ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true, executionTaskQueue)); + boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) + && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); + + final ProcessWorkUnit processWorkUnitStub; + if (dynamicScalingEnabled) { + // Route to execution queue for specialized worker + String executionTaskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, + ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true, executionTaskQueue)); + } else { + // Use default queue (inherits from workflow) + processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, + ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true)); + } return Async.function(processWorkUnitStub::processWorkUnit, wu); } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index d72f39da0ca..2a211157ddf 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -98,43 +98,32 @@ public DynamicScalingYarnService(Config config, String applicationName, String a } /** - * Initializes stage-specific worker profiles for dynamic scaling mode. - * Creates two profiles derived from baseline: - * 1. DiscoveryCommitWorker - for discovery/commit activities - * 2. ExecutionWorker - for execution activities + * Initializes stage-specific worker profile for dynamic scaling mode. + * Creates ExecutionWorker profile for processing work units. + * Default WorkFulfillmentWorker handles all other activities (discovery, commit, etc). */ private void initializeDynamicScalingProfiles() { - log.info("Initializing stage-specific profiles"); + log.info("Initializing execution worker profile for dynamic scaling"); long currTimeMillis = System.currentTimeMillis(); List initialDirectives = new ArrayList<>(); - // Container 1: DiscoveryCommitWorker (for discovery/commit activities) - ProfileOverlay discoveryCommitOverlay = createDiscoveryCommitProfileOverlay(); - initialDirectives.add(new ScalingDirective( - "initial-discovery-commit", - 1, // setPoint = 1 container - currTimeMillis, - WorkforceProfiles.BASELINE_NAME, - discoveryCommitOverlay - )); - - // Container 2: ExecutionWorker (for execution activities) + // ExecutionWorker for execution activities only ProfileOverlay executionOverlay = createExecutionProfileOverlay(); initialDirectives.add(new ScalingDirective( "initial-execution", 1, // setPoint = 1 container - currTimeMillis + EPSILON_MIILIS, + currTimeMillis, WorkforceProfiles.BASELINE_NAME, executionOverlay )); // Apply initial directives to workforce plan this.workforcePlan.reviseWhenNewer(initialDirectives, ire -> { - log.error("Failed to create stage-specific profiles", ire); - throw new RuntimeException("Failed to initialize stage-specific profiles for dynamic scaling", ire); + log.error("Failed to create execution worker profile", ire); + throw new RuntimeException("Failed to initialize execution worker profile for dynamic scaling", ire); }); - log.info("Initialized {} stage-specific profiles for dynamic scaling", initialDirectives.size()); + log.info("Initialized execution worker profile for dynamic scaling"); } @Override @@ -145,36 +134,6 @@ protected synchronized void requestInitialContainers() { requestNewContainersForStaffingDeltas(deltas); } - private ProfileOverlay createDiscoveryCommitProfileOverlay() { - List overlayPairs = new ArrayList<>(); - - // Set worker class - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker" - )); - - // Set Helix tag - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "discovery-commit" - )); - - // Set stage-specific memory (discovery/commit operations are typically lightweight) - // Falls back to global CONTAINER_MEMORY_MBS_KEY if stage-specific memory not configured - if (this.config.hasPath(GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB)) { - String discoveryCommitMemoryMb = this.config.getString( - GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_MEMORY_MB); - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - discoveryCommitMemoryMb - )); - log.info("Setting discovery-commit container memory to {} MB", discoveryCommitMemoryMb); - } - - return new ProfileOverlay.Adding(overlayPairs); - } - private ProfileOverlay createExecutionProfileOverlay() { List overlayPairs = new ArrayList<>(); @@ -399,50 +358,30 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont /** * Gets the memory multiplier for OOM retries for a specific workflow stage. + * Uses execution-specific config for WORK_EXECUTION, default for all others. */ private int getMemoryMultiplierForStage(WorkflowStage stage) { - String key; - int defaultValue; - - switch (stage) { - case WORK_DISCOVERY: - case COMMIT: - key = GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER; - defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_OOM_MEMORY_MULTIPLIER; - break; - case WORK_EXECUTION: - key = GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; - defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; - break; - default: - return DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER; + if (stage == WorkflowStage.WORK_EXECUTION) { + return config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER) + ? config.getInt(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER) + : GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; } - - return config.hasPath(key) ? config.getInt(key) : defaultValue; + // Use default multiplier for all non-execution stages (discovery, commit, etc.) + return DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER; } /** * Gets the maximum memory allowed for OOM retries for a specific workflow stage. + * Uses execution-specific config for WORK_EXECUTION, default for all others. */ private int getMaxMemoryForStage(WorkflowStage stage) { - String key; - int defaultValue; - - switch (stage) { - case WORK_DISCOVERY: - case COMMIT: - key = GobblinTemporalConfigurationKeys.DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB; - defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_DISCOVERY_COMMIT_OOM_MAX_MEMORY_MB; - break; - case WORK_EXECUTION: - key = GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB; - defaultValue = GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB; - break; - default: - return MAX_REPLACEMENT_CONTAINER_MEMORY_MBS; + if (stage == WorkflowStage.WORK_EXECUTION) { + return config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB) + ? config.getInt(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB) + : GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB; } - - return config.hasPath(key) ? config.getInt(key) : defaultValue; + // Use default max memory for all non-execution stages (discovery, commit, etc.) + return MAX_REPLACEMENT_CONTAINER_MEMORY_MBS; } /** @@ -470,18 +409,14 @@ private ProfileOverlay createStageSpecificOOMOverlay(WorkflowStage stage, int ne /** * Gets the worker class for a specific workflow stage. + * Returns ExecutionWorker for WORK_EXECUTION, WorkFulfillmentWorker for all others. */ private String getWorkerClassForStage(WorkflowStage stage) { - switch (stage) { - case WORK_DISCOVERY: - return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; - case WORK_EXECUTION: - return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; - case COMMIT: - return "org.apache.gobblin.temporal.ddm.worker.DiscoveryCommitWorker"; - default: - return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; + if (stage == WorkflowStage.WORK_EXECUTION) { + return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; } + // All non-execution stages use WorkFulfillmentWorker (discovery, commit, etc.) + return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; } } From a428b4b77ce0db9633ef71ad61b106a417582914 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Fri, 12 Dec 2025 13:29:59 +0530 Subject: [PATCH 03/13] code refactoring --- conf/yarn/application.conf | 19 +----- .../GobblinTemporalConfigurationKeys.java | 6 +- ...tractRecommendScalingForWorkUnitsImpl.java | 63 +++++++------------ .../impl/ExecuteGobblinWorkflowImpl.java | 7 +++ .../temporal/dynamic/WorkforcePlan.java | 8 +-- .../yarn/DynamicScalingYarnService.java | 54 ++++------------ 6 files changed, 42 insertions(+), 115 deletions(-) diff --git a/conf/yarn/application.conf b/conf/yarn/application.conf index 61c27f09125..b7557c83402 100644 --- a/conf/yarn/application.conf +++ b/conf/yarn/application.conf @@ -97,21 +97,4 @@ job.execinfo.server.enabled=false job.history.store.enabled=false -# Work Discovery stage -gobblin.temporal.stage.workDiscovery.memory.mb=16384 -gobblin.temporal.stage.workDiscovery.oom.memory.multiplier=2 -gobblin.temporal.stage.workDiscovery.oom.max.memory.mb=131072 - -# Work Execution stage -gobblin.temporal.stage.workExecution.memory.mb=8192 -gobblin.temporal.stage.workExecution.oom.memory.multiplier=2 -gobblin.temporal.stage.workExecution.oom.max.memory.mb=65536 - -# Commit stage -gobblin.temporal.stage.commit.memory.mb=4096 -gobblin.temporal.stage.commit.oom.memory.multiplier=2 -gobblin.temporal.stage.commit.oom.max.memory.mb=32768 - -gobblin.temporal.discovery.task.queue.name=GobblinTemporalDiscoveryQueue -gobblin.temporal.execution.task.queue.name=GobblinTemporalExecutionQueue -gobblin.temporal.commit.task.queue.name=GobblinTemporalCommitQueue +gobblin.temporal.execution.task.queue.name=GobblinTemporalExecutionQueue \ No newline at end of file diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index 6411862e3fb..7213b552eda 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -136,15 +136,11 @@ public interface GobblinTemporalConfigurationKeys { /** * Stage-specific resource configuration for execution workers - * Allows independent memory and OOM configuration for work execution stage + * Allows independent memory configuration for work execution stage when dynamic scaling is enabled */ String STAGE_SPECIFIC_PREFIX = PREFIX + "stage."; // Work Execution stage configuration (for ExecutionWorker when dynamic scaling is enabled) String WORK_EXECUTION_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.memory.mb"; - String WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = STAGE_SPECIFIC_PREFIX + "workExecution.oom.memory.multiplier"; - int DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER = 2; - String WORK_EXECUTION_OOM_MAX_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.oom.max.memory.mb"; - int DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB = 65536; // 64GB (memory-intensive operations) } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java index 7d2532041f0..89dcd7fa719 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java @@ -49,84 +49,63 @@ public abstract class AbstractRecommendScalingForWorkUnitsImpl implements Recomm @Override public List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps, WorkflowStage stage) { + // NOTE: Scaling is only done for WORK_EXECUTION stage (processing work units) // NOTE: no attempt to determine the current scaling - per `RecommendScalingForWorkUnits` javadoc, the `ScalingDirective`(s) returned must "stand alone", // presuming nothing of the current `WorkforcePlan`'s `WorkforceStaffing` JobState jobState = new JobState(jobProps); ScalingDirective procWUsWorkerScaling = new ScalingDirective( - calcProfileDerivationName(jobState, stage), + calcProfileDerivationName(jobState), calcDerivationSetPoint(remainingWork, sourceClass, timeBudget, jobState), System.currentTimeMillis(), - Optional.of(calcProfileDerivation(calcBasisProfileName(jobState, stage), remainingWork, sourceClass, jobState, stage)) + Optional.of(calcProfileDerivation(calcBasisProfileName(jobState), remainingWork, sourceClass, jobState)) ); - log.info("Recommended re-scaling for {} stage to process work units: {}", stage, procWUsWorkerScaling); + log.info("Recommended re-scaling for WORK_EXECUTION to process work units: {}", procWUsWorkerScaling); return Arrays.asList(procWUsWorkerScaling); } protected abstract int calcDerivationSetPoint(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, JobState jobState); - protected ProfileDerivation calcProfileDerivation(String basisProfileName, WorkUnitsSizeSummary remainingWork, String sourceClass, JobState jobState, WorkflowStage stage) { - // Create overlay with stage-specific memory and worker class - ProfileOverlay overlay = createStageSpecificOverlay(jobState, stage); + protected ProfileDerivation calcProfileDerivation(String basisProfileName, WorkUnitsSizeSummary remainingWork, String sourceClass, JobState jobState) { + // Create overlay with execution-specific memory and worker class + ProfileOverlay overlay = createExecutionWorkerOverlay(jobState); return new ProfileDerivation(basisProfileName, overlay); } - protected String calcProfileDerivationName(JobState jobState, WorkflowStage stage) { + protected String calcProfileDerivationName(JobState jobState) { // TODO: if we ever return > 1 directive, append a monotonically increasing number to avoid collisions - return stage.getProcessingProfileName(); // e.g., "workExecution-proc" + return WorkflowStage.WORK_EXECUTION.getProcessingProfileName(); // "workExecution-proc" } - protected String calcBasisProfileName(JobState jobState, WorkflowStage stage) { + protected String calcBasisProfileName(JobState jobState) { // Always derive from the global baseline return WorkforceProfiles.BASELINE_NAME; } /** - * Creates a ProfileOverlay with stage-specific memory and worker class configuration. - * This allows deriving stage-specific profiles from the global baseline. + * Creates a ProfileOverlay for ExecutionWorker with optional memory override and worker class. + * + * This overlay is applied to the baseline profile to create execution worker containers. + * If gobblin.temporal.stage.workExecution.memory.mb is configured in job properties, + * it overrides the baseline container memory for execution workers. */ - private ProfileOverlay createStageSpecificOverlay(JobState jobState, WorkflowStage stage) { + private ProfileOverlay createExecutionWorkerOverlay(JobState jobState) { List overlayPairs = new java.util.ArrayList<>(); - // Add stage-specific memory if configured - String memoryKey = getStageMemoryConfigKey(stage); - if (jobState.contains(memoryKey)) { + // Add execution-specific memory if configured (overrides baseline memory) + if (jobState.contains(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB)) { overlayPairs.add(new ProfileOverlay.KVPair( GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - jobState.getProp(memoryKey) + jobState.getProp(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB) )); } - // Add stage-specific worker class - String workerClass = getWorkerClassForStage(stage); + // Add ExecutionWorker class to ensure correct task queue routing overlayPairs.add(new ProfileOverlay.KVPair( GobblinTemporalConfigurationKeys.WORKER_CLASS, - workerClass + "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker" )); return overlayPairs.isEmpty() ? ProfileOverlay.unchanged() : new ProfileOverlay.Adding(overlayPairs); } - /** - * Returns the configuration key for stage-specific memory. - * Only WORK_EXECUTION has stage-specific memory config; others use default. - */ - private String getStageMemoryConfigKey(WorkflowStage stage) { - if (stage == WorkflowStage.WORK_EXECUTION) { - return GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB; - } - // Non-execution stages (discovery, commit) use default/baseline memory - return null; - } - - /** - * Gets the worker class for a specific workflow stage. - * Returns ExecutionWorker for WORK_EXECUTION, WorkFulfillmentWorker for all others. - */ - private String getWorkerClassForStage(WorkflowStage stage) { - if (stage == WorkflowStage.WORK_EXECUTION) { - return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; - } - // All non-execution stages use WorkFulfillmentWorker (discovery, commit, etc.) - return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; - } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 008aec54795..4f78b62c431 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.gobblin.temporal.ddm.activity.ProcessWorkUnit; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -165,11 +166,17 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event } protected ProcessWorkUnitsWorkflow createProcessWorkUnitsWorkflow(Properties jobProps) { + com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(jobProps); + boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) + && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); + ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_TERMINATE) .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(PROCESS_WORKFLOW_ID_BASE, ConfigFactory.parseProperties(jobProps))) .setSearchAttributes(TemporalWorkFlowUtils.generateGaasSearchAttributes(jobProps)) + .setTaskQueue(dynamicScalingEnabled ? WorkflowStage.WORK_EXECUTION.getTaskQueue(config) : null) .build(); + return Workflow.newChildWorkflowStub(ProcessWorkUnitsWorkflow.class, childOpts); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index 94b2dd4fa29..b08ad51f722 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -84,13 +84,7 @@ protected UnrecognizedProfile(ScalingDirective directive) { /** create new plan with the initial, baseline worker profile using `baselineConfig` at `initialSetPoint` */ public WorkforcePlan(Config baselineConfig, int initialSetPoint) { this.profiles = WorkforceProfiles.withBaseline(baselineConfig); - this.staffing = WorkforceStaffing.initialize(0); - // Initial containers use the global baseline profile - this.staffing.reviseStaffing( - WorkforceProfiles.BASELINE_NAME, - initialSetPoint, - 0 - ); + this.staffing = WorkforceStaffing.initialize(initialSetPoint); this.lastRevisionEpochMillis = 0; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 2a211157ddf..542c6b8b7c9 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -326,24 +326,20 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont scalingDirectives.add(new ScalingDirective(workerProfile.getName(), currNumContainers - 1, currTimeMillis)); } - // Get stage-specific OOM configuration - int memoryMultiplier = getMemoryMultiplierForStage(stage); - int maxMemoryMbs = getMaxMemoryForStage(stage); - - // Request a replacement container with stage-specific limits + // Use default OOM configuration for all containers int currContainerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - if (currContainerMemoryMbs >= maxMemoryMbs) { - log.warn("Container {} for stage {} already had max allowed memory {} MBs (stage max: {} MBs). Not requesting a replacement container.", - completedContainerId, stage, currContainerMemoryMbs, maxMemoryMbs); + if (currContainerMemoryMbs >= MAX_REPLACEMENT_CONTAINER_MEMORY_MBS) { + log.warn("Container {} already had max allowed memory {} MBs (max: {} MBs). Not requesting a replacement container.", + completedContainerId, currContainerMemoryMbs, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); return; } - int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * memoryMultiplier, maxMemoryMbs); + int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); - log.info("Creating OOM replacement for stage {} with memory: {} MB -> {} MB (max: {} MB)", - stage, currContainerMemoryMbs, newContainerMemoryMbs, maxMemoryMbs); + log.info("Creating OOM replacement with memory: {} MB -> {} MB (max: {} MB)", + currContainerMemoryMbs, newContainerMemoryMbs, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); - // Derive from global baseline with stage-specific overlays - ProfileOverlay overlay = createStageSpecificOOMOverlay(stage, newContainerMemoryMbs); + // Derive from global baseline with memory and worker class overlays + ProfileOverlay overlay = createOOMReplacementOverlay(stage, newContainerMemoryMbs); Optional optProfileDerivation = Optional.of( new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, overlay) ); @@ -357,38 +353,10 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont } /** - * Gets the memory multiplier for OOM retries for a specific workflow stage. - * Uses execution-specific config for WORK_EXECUTION, default for all others. - */ - private int getMemoryMultiplierForStage(WorkflowStage stage) { - if (stage == WorkflowStage.WORK_EXECUTION) { - return config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER) - ? config.getInt(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MEMORY_MULTIPLIER) - : GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MEMORY_MULTIPLIER; - } - // Use default multiplier for all non-execution stages (discovery, commit, etc.) - return DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER; - } - - /** - * Gets the maximum memory allowed for OOM retries for a specific workflow stage. - * Uses execution-specific config for WORK_EXECUTION, default for all others. - */ - private int getMaxMemoryForStage(WorkflowStage stage) { - if (stage == WorkflowStage.WORK_EXECUTION) { - return config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB) - ? config.getInt(GobblinTemporalConfigurationKeys.WORK_EXECUTION_OOM_MAX_MEMORY_MB) - : GobblinTemporalConfigurationKeys.DEFAULT_WORK_EXECUTION_OOM_MAX_MEMORY_MB; - } - // Use default max memory for all non-execution stages (discovery, commit, etc.) - return MAX_REPLACEMENT_CONTAINER_MEMORY_MBS; - } - - /** - * Creates a ProfileOverlay for OOM replacement with stage-specific memory and worker class. + * Creates a ProfileOverlay for OOM replacement with increased memory and appropriate worker class. * This derives from the global baseline, ensuring task queue routing is preserved. */ - private ProfileOverlay createStageSpecificOOMOverlay(WorkflowStage stage, int newMemoryMbs) { + private ProfileOverlay createOOMReplacementOverlay(WorkflowStage stage, int newMemoryMbs) { List overlayPairs = new ArrayList<>(); // Add increased memory From 7b9862158a8af78260e63d47d41259d1f6e6fd5e Mon Sep 17 00:00:00 2001 From: agam-99 Date: Fri, 12 Dec 2025 14:01:18 +0530 Subject: [PATCH 04/13] code cleanup --- .../yarn/DynamicScalingYarnService.java | 68 ++++--------------- 1 file changed, 12 insertions(+), 56 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 542c6b8b7c9..1ca3a5393b0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -306,16 +306,12 @@ private void handleAbortedContainer(ContainerId completedContainerId, ContainerI } private synchronized void handleContainerExitedWithOOM(ContainerId completedContainerId, ContainerInfo completedContainerInfo) { - WorkerProfile workerProfile = completedContainerInfo.getWorkerProfile(); - - // Determine which workflow stage this container belongs to - WorkflowStage stage = WorkflowStage.fromProfileName(workerProfile.getName()); - - log.info("Container {} for profile {} (stage: {}) exited with OOM, starting to launch a replacement container", - completedContainerId, completedContainerInfo.getWorkerProfileName(), stage); + log.info("Container {} for profile {} exited with OOM, starting to launch a replacement container", + completedContainerId, completedContainerInfo.getWorkerProfileName()); List scalingDirectives = new ArrayList<>(); + WorkerProfile workerProfile = completedContainerInfo.getWorkerProfile(); long currTimeMillis = System.currentTimeMillis(); // Update the current staffing to reflect the container that exited with OOM int currNumContainers = this.actualWorkforceStaffing.getStaffing(workerProfile.getName()).orElse(0); @@ -326,25 +322,20 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont scalingDirectives.add(new ScalingDirective(workerProfile.getName(), currNumContainers - 1, currTimeMillis)); } - // Use default OOM configuration for all containers + // Request a replacement container int currContainerMemoryMbs = workerProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); if (currContainerMemoryMbs >= MAX_REPLACEMENT_CONTAINER_MEMORY_MBS) { - log.warn("Container {} already had max allowed memory {} MBs (max: {} MBs). Not requesting a replacement container.", - completedContainerId, currContainerMemoryMbs, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); + log.warn("Container {} already had max allowed memory {} MBs. Not requesting a replacement container.", + completedContainerId, currContainerMemoryMbs); return; } - int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); - - log.info("Creating OOM replacement with memory: {} MB -> {} MB (max: {} MB)", - currContainerMemoryMbs, newContainerMemoryMbs, MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); - - // Derive from global baseline with memory and worker class overlays - ProfileOverlay overlay = createOOMReplacementOverlay(stage, newContainerMemoryMbs); - Optional optProfileDerivation = Optional.of( - new ProfileDerivation(WorkforceProfiles.BASELINE_NAME, overlay) - ); + int newContainerMemoryMbs = Math.min(currContainerMemoryMbs * DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER, + MAX_REPLACEMENT_CONTAINER_MEMORY_MBS); + Optional optProfileDerivation = Optional.of(new ProfileDerivation(workerProfile.getName(), + new ProfileOverlay.Adding(new ProfileOverlay.KVPair(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, newContainerMemoryMbs + "")) + )); scalingDirectives.add(new ScalingDirective( - stage.getProfileBaseName() + "-oomReplacement-" + profileNameSuffixGenerator.getAndIncrement(), + DEFAULT_REPLACEMENT_CONTAINER_WORKER_PROFILE_NAME_PREFIX + "-" + profileNameSuffixGenerator.getAndIncrement(), 1, currTimeMillis + EPSILON_MIILIS, // Each scaling directive should have a newer timestamp than the previous one optProfileDerivation @@ -352,39 +343,4 @@ private synchronized void handleContainerExitedWithOOM(ContainerId completedCont reviseWorkforcePlanAndRequestNewContainers(scalingDirectives); } - /** - * Creates a ProfileOverlay for OOM replacement with increased memory and appropriate worker class. - * This derives from the global baseline, ensuring task queue routing is preserved. - */ - private ProfileOverlay createOOMReplacementOverlay(WorkflowStage stage, int newMemoryMbs) { - List overlayPairs = new ArrayList<>(); - - // Add increased memory - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - String.valueOf(newMemoryMbs) - )); - - // Add stage-specific worker class to ensure correct task queue routing - String workerClass = getWorkerClassForStage(stage); - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - workerClass - )); - - return new ProfileOverlay.Adding(overlayPairs); - } - - /** - * Gets the worker class for a specific workflow stage. - * Returns ExecutionWorker for WORK_EXECUTION, WorkFulfillmentWorker for all others. - */ - private String getWorkerClassForStage(WorkflowStage stage) { - if (stage == WorkflowStage.WORK_EXECUTION) { - return "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"; - } - // All non-execution stages use WorkFulfillmentWorker (discovery, commit, etc.) - return "org.apache.gobblin.temporal.ddm.worker.WorkFulfillmentWorker"; - } - } From af5e22baceb3e7802dd45cd78822b15d036de9bf Mon Sep 17 00:00:00 2001 From: agam-99 Date: Sat, 13 Dec 2025 09:53:05 +0530 Subject: [PATCH 05/13] fix: spin up 1 container initially by default --- .../temporal/yarn/DynamicScalingYarnService.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 1ca3a5393b0..a5ba5d81de7 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -69,7 +69,6 @@ public class DynamicScalingYarnService extends YarnService { private final WorkforceStaffing actualWorkforceStaffing; /** this holds the current total workforce plan as per latest received scaling directives */ private final WorkforcePlan workforcePlan; - private final int initialContainers; protected final Queue removedContainerIds; private final AtomicLong profileNameSuffixGenerator; private final boolean dynamicScalingEnabled; @@ -80,18 +79,13 @@ public DynamicScalingYarnService(Config config, String applicationName, String a this.dynamicScalingEnabled = ConfigUtils.getBoolean(config, GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, false); - this.initialContainers = this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY); this.actualWorkforceStaffing = WorkforceStaffing.initialize(0); - // Initialize workforce plan: - // - For dynamic scaling: start with 0 baseline, then add stage-specific profiles - // - For traditional mode: initialize baseline with configured initial containers - int baselineSetPoint = this.dynamicScalingEnabled ? 0 : this.initialContainers; - this.workforcePlan = new WorkforcePlan(this.config, baselineSetPoint); + this.workforcePlan = new WorkforcePlan(this.config, this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY)); + this.removedContainerIds = new ConcurrentLinkedQueue<>(); this.profileNameSuffixGenerator = new AtomicLong(); - // For dynamic scaling, add stage-specific profiles derived from baseline if (this.dynamicScalingEnabled) { initializeDynamicScalingProfiles(); } From c6f8647a455b6bab91ebd0914cb4213d72e6c956 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Sat, 13 Dec 2025 12:23:59 +0530 Subject: [PATCH 06/13] proper queue routing for commit phase --- .../workflow/impl/ProcessWorkUnitsWorkflowImpl.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java index 829d0aa8798..19e72dce3c3 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java @@ -22,6 +22,7 @@ import lombok.extern.slf4j.Slf4j; +import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; import io.temporal.api.enums.v1.ParentClosePolicy; @@ -30,6 +31,7 @@ import io.temporal.workflow.Workflow; import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.cluster.WorkerConfig; import org.apache.gobblin.temporal.ddm.util.TemporalWorkFlowUtils; import org.apache.gobblin.temporal.ddm.work.CommitStats; @@ -144,12 +146,17 @@ protected NestingExecWorkflow createProcessingWorkflow(FileS } protected CommitStepWorkflow createCommitStepWorkflow(Map searchAttributes) { + Config config = WorkerConfig.of(this).orElse(ConfigFactory.empty()); + String defaultTaskQueue = config.hasPath(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE) + ? config.getString(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE) + : GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE; + ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() // TODO: verify to instead use: Policy.PARENT_CLOSE_POLICY_TERMINATE) .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_ABANDON) .setSearchAttributes(searchAttributes) - .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, - WorkerConfig.of(this).orElse(ConfigFactory.empty()))) + .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, config)) + .setTaskQueue(defaultTaskQueue) .build(); return Workflow.newChildWorkflowStub(CommitStepWorkflow.class, childOpts); From 4bdc43a20899701067f41ad327d4c4cb5761e46d Mon Sep 17 00:00:00 2001 From: agam-99 Date: Sun, 14 Dec 2025 12:03:17 +0530 Subject: [PATCH 07/13] code cleanup --- conf/yarn/application.conf | 5 +- .../GobblinTemporalConfigurationKeys.java | 13 ++--- .../cluster/AbstractTemporalWorker.java | 4 -- .../cluster/GobblinTemporalTaskRunner.java | 7 +-- .../temporal/ddm/activity/ActivityType.java | 13 ++--- .../RecommendScalingForWorkUnits.java | 4 +- ...tractRecommendScalingForWorkUnitsImpl.java | 16 ++---- ...calingForWorkUnitsLinearHeuristicImpl.java | 4 +- .../temporal/ddm/worker/ExecutionWorker.java | 13 +++-- .../ddm/worker/WorkFulfillmentWorker.java | 18 +------ .../temporal/ddm/workflow/WorkflowStage.java | 51 ++----------------- .../workflow/impl/CommitStepWorkflowImpl.java | 3 +- .../impl/ExecuteGobblinWorkflowImpl.java | 21 ++++---- ...tingExecOfProcessWorkUnitWorkflowImpl.java | 20 +------- .../impl/ProcessWorkUnitsWorkflowImpl.java | 11 +--- .../yarn/DynamicScalingYarnService.java | 25 +++------ .../gobblin/temporal/yarn/YarnService.java | 7 --- 17 files changed, 46 insertions(+), 189 deletions(-) diff --git a/conf/yarn/application.conf b/conf/yarn/application.conf index b7557c83402..97661ebd888 100644 --- a/conf/yarn/application.conf +++ b/conf/yarn/application.conf @@ -94,7 +94,4 @@ admin.server.enabled=false # job history store ( WARN [GobblinYarnAppLauncher] NOT starting the admin UI because the job execution info server is NOT enabled ) job.execinfo.server.enabled=false -job.history.store.enabled=false - - -gobblin.temporal.execution.task.queue.name=GobblinTemporalExecutionQueue \ No newline at end of file +job.history.store.enabled=false \ No newline at end of file diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index 7213b552eda..c194b2e76d4 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -18,6 +18,7 @@ package org.apache.gobblin.temporal; import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.temporal.ddm.worker.ExecutionWorker; import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldJobLauncher; import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldWorker; @@ -29,16 +30,18 @@ public interface GobblinTemporalConfigurationKeys { String PREFIX = "gobblin.temporal."; + String STAGE_SPECIFIC_PREFIX = PREFIX + "stage."; String WORKER_CLASS = PREFIX + "worker.class"; String DEFAULT_WORKER_CLASS = HelloWorldWorker.class.getName(); + String EXECUTION_WORKER_CLASS = ExecutionWorker.class.getName(); String GOBBLIN_TEMPORAL_NAMESPACE = PREFIX + "namespace"; String DEFAULT_GOBBLIN_TEMPORAL_NAMESPACE = PREFIX + "namespace"; String GOBBLIN_TEMPORAL_TASK_QUEUE = PREFIX + "task.queue.name"; String DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue"; - // Execution task queue for worker specialization (when dynamic scaling is enabled) + // Execution task queue for work execution specialization String EXECUTION_TASK_QUEUE = PREFIX + "execution.task.queue.name"; String DEFAULT_EXECUTION_TASK_QUEUE = "GobblinTemporalExecutionQueue"; String GOBBLIN_TEMPORAL_JOB_LAUNCHER_PREFIX = PREFIX + "job.launcher."; @@ -134,13 +137,5 @@ public interface GobblinTemporalConfigurationKeys { String TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = TEMPORAL_ACTIVITY_RETRY_OPTIONS + "maximum.attempts"; int DEFAULT_TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = 4; - /** - * Stage-specific resource configuration for execution workers - * Allows independent memory configuration for work execution stage when dynamic scaling is enabled - */ - String STAGE_SPECIFIC_PREFIX = PREFIX + "stage."; - - // Work Execution stage configuration (for ExecutionWorker when dynamic scaling is enabled) String WORK_EXECUTION_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.memory.mb"; - } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java index 26f1e68fb49..b569417f0b8 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/AbstractTemporalWorker.java @@ -73,10 +73,6 @@ protected WorkerOptions createWorkerOptions() { /** @return activity instances; NOTE: activities must be stateless and thread-safe, so a shared instance is used. */ protected abstract Object[] getActivityImplInstances(); - /** - * @return the task queue name this worker should poll from. - * Subclasses can override this to specify a custom task queue. - */ protected String getTaskQueue() { return ConfigUtils.getString(config, GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java index 3621eb3edb0..6bc393b6976 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java @@ -250,11 +250,8 @@ private TemporalWorker initiateWorker() throws Exception { WorkflowClient client = TemporalWorkflowClientFactory.createClientInstance( managedWorkflowServiceStubs.getWorkflowServiceStubs(), namespace); - // Read worker class from system property (set by YarnService when launching containers) - // Fallback to config for non-YARN deployments - String workerClassName = System.getProperty(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigUtils.getString(clusterConfig, GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); + String workerClassName = ConfigUtils.getString(clusterConfig, GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS); logger.info("Creating worker - class: '{}'", workerClassName); Config workerConfig = clusterConfig; TemporalWorker worker = GobblinConstructorUtils.invokeLongestConstructor( diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java index b62731c6caa..a283c179d41 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java @@ -72,23 +72,16 @@ public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbe .build(); } - /** - * Build activity options with a specific task queue for stage-specific routing. - * @param props job properties - * @param setHeartbeatTimeout whether to set heartbeat timeout - * @param taskQueue the task queue to route this activity to - * @return ActivityOptions with task queue set - */ public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbeatTimeout, String taskQueue) { ActivityOptions.Builder builder = ActivityOptions.newBuilder() .setStartToCloseTimeout(getStartToCloseTimeout(props)) .setRetryOptions(buildRetryOptions(props)) - .setTaskQueue(taskQueue); // Route to specific queue - + .setTaskQueue(taskQueue); + if (setHeartbeatTimeout) { builder.setHeartbeatTimeout(getHeartbeatTimeout(props)); } - + return builder.build(); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java index 696a4249ab7..35ba0553efe 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java @@ -25,7 +25,6 @@ import org.apache.gobblin.source.workunit.WorkUnit; import org.apache.gobblin.temporal.ddm.work.TimeBudget; import org.apache.gobblin.temporal.ddm.work.WorkUnitsSizeSummary; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.dynamic.ScalingDirective; @@ -48,9 +47,8 @@ public interface RecommendScalingForWorkUnits { * @param sourceClass contextualizes the `WorkUnitsSizeSummary` and should name a {@link org.apache.gobblin.source.Source} * @param timeBudget the remaining target duration for processing the summarized `WorkUnit`s * @param jobProps all job props, to either guide the recommendation or better contextualize the nature of the `remainingWork` - * @param stage the workflow stage for which scaling is being recommended (e.g., WORK_EXECUTION) * @return the {@link ScalingDirective}s to process the summarized {@link WorkUnit}s within {@link TimeBudget} */ @ActivityMethod - List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps, WorkflowStage stage); + List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps); } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java index 89dcd7fa719..78363917ab0 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java @@ -48,8 +48,7 @@ public abstract class AbstractRecommendScalingForWorkUnitsImpl implements Recomm public static final String DEFAULT_PROFILE_DERIVATION_NAME = "workUnitsProc"; @Override - public List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps, WorkflowStage stage) { - // NOTE: Scaling is only done for WORK_EXECUTION stage (processing work units) + public List recommendScaling(WorkUnitsSizeSummary remainingWork, String sourceClass, TimeBudget timeBudget, Properties jobProps) { // NOTE: no attempt to determine the current scaling - per `RecommendScalingForWorkUnits` javadoc, the `ScalingDirective`(s) returned must "stand alone", // presuming nothing of the current `WorkforcePlan`'s `WorkforceStaffing` JobState jobState = new JobState(jobProps); @@ -59,7 +58,7 @@ public List recommendScaling(WorkUnitsSizeSummary remainingWor System.currentTimeMillis(), Optional.of(calcProfileDerivation(calcBasisProfileName(jobState), remainingWork, sourceClass, jobState)) ); - log.info("Recommended re-scaling for WORK_EXECUTION to process work units: {}", procWUsWorkerScaling); + log.info("Recommended re-scaling to process work units: {}", procWUsWorkerScaling); return Arrays.asList(procWUsWorkerScaling); } @@ -73,7 +72,7 @@ protected ProfileDerivation calcProfileDerivation(String basisProfileName, WorkU protected String calcProfileDerivationName(JobState jobState) { // TODO: if we ever return > 1 directive, append a monotonically increasing number to avoid collisions - return WorkflowStage.WORK_EXECUTION.getProcessingProfileName(); // "workExecution-proc" + return DEFAULT_PROFILE_DERIVATION_NAME; } protected String calcBasisProfileName(JobState jobState) { @@ -81,13 +80,6 @@ protected String calcBasisProfileName(JobState jobState) { return WorkforceProfiles.BASELINE_NAME; } - /** - * Creates a ProfileOverlay for ExecutionWorker with optional memory override and worker class. - * - * This overlay is applied to the baseline profile to create execution worker containers. - * If gobblin.temporal.stage.workExecution.memory.mb is configured in job properties, - * it overrides the baseline container memory for execution workers. - */ private ProfileOverlay createExecutionWorkerOverlay(JobState jobState) { List overlayPairs = new java.util.ArrayList<>(); @@ -102,7 +94,7 @@ private ProfileOverlay createExecutionWorkerOverlay(JobState jobState) { // Add ExecutionWorker class to ensure correct task queue routing overlayPairs.add(new ProfileOverlay.KVPair( GobblinTemporalConfigurationKeys.WORKER_CLASS, - "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker" + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS )); return overlayPairs.isEmpty() ? ProfileOverlay.unchanged() : new ProfileOverlay.Adding(overlayPairs); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java index 55b05484488..777f7e9c200 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/RecommendScalingForWorkUnitsLinearHeuristicImpl.java @@ -97,9 +97,7 @@ protected int calcDerivationSetPoint(WorkUnitsSizeSummary remainingWork, String int recommendedNumContainers = Math.max(numContainerForWUs, numContainerForThroughout); log.info("Recommended auto-scaling: {} containers, no. of containers considering throughput: {}, no. of containers considering WUs: {}", recommendedNumContainers, numContainerForThroughout, numContainerForWUs); -/** - * - */ + return recommendedNumContainers; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java index fb7dc41718b..3a2e571f033 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java @@ -36,7 +36,7 @@ * Specialized worker for Work Execution stage. * This worker only registers activities for: * - ProcessWorkUnit (Work Execution) - * + * * Runs on containers with stage-specific memory for work execution operations. * Polls the execution task queue to ensure activities run on appropriately-sized containers. */ @@ -52,17 +52,16 @@ public ExecutionWorker(Config config, WorkflowClient workflowClient) { @Override protected Class[] getWorkflowImplClasses() { - return new Class[] { - ProcessWorkUnitsWorkflowImpl.class, - NestingExecOfProcessWorkUnitWorkflowImpl.class + return new Class[] { + ProcessWorkUnitsWorkflowImpl.class, + NestingExecOfProcessWorkUnitWorkflowImpl.class }; } @Override protected Object[] getActivityImplInstances() { - // Only register activities for Execution stage - return new Object[] { - new ProcessWorkUnitImpl() // Work Execution only + return new Object[] { + new ProcessWorkUnitImpl() }; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java index 9d860308af3..12fe6c4d849 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/WorkFulfillmentWorker.java @@ -17,19 +17,15 @@ package org.apache.gobblin.temporal.ddm.worker; -import java.util.Arrays; import java.util.concurrent.TimeUnit; import com.typesafe.config.Config; import io.temporal.client.WorkflowClient; -import io.temporal.worker.Worker; -import io.temporal.worker.WorkerFactory; import io.temporal.worker.WorkerOptions; import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.cluster.AbstractTemporalWorker; -import org.apache.gobblin.temporal.cluster.WorkerConfig; import org.apache.gobblin.temporal.ddm.activity.impl.CommitActivityImpl; import org.apache.gobblin.temporal.ddm.activity.impl.DeleteWorkDirsActivityImpl; import org.apache.gobblin.temporal.ddm.activity.impl.GenerateWorkUnitsImpl; @@ -44,19 +40,7 @@ import org.apache.gobblin.util.ConfigUtils; -/** - * Unified worker that handles ALL workflow stages except execution (Discovery, Commit, etc.). - * - * This worker polls the DEFAULT task queue and is used: - * - Always for discovery, commit, and other non-execution activities - * - For execution activities when dynamic scaling is disabled - * - * When dynamic scaling is enabled, execution activities are routed to the execution queue - * and handled by the specialized ExecutionWorker. - * - * This prevents resource mismatches (e.g., execution tasks requiring 64GB running - * on a baseline container with only 8GB). - */ +/** Worker for the {@link ProcessWorkUnitsWorkflowImpl} super-workflow */ public class WorkFulfillmentWorker extends AbstractTemporalWorker { public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; // TODO: make configurable! public int maxExecutionConcurrency; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java index 82125f51ffd..fcc17f1004d 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStage.java @@ -17,13 +17,14 @@ package org.apache.gobblin.temporal.ddm.workflow; +import com.typesafe.config.Config; import lombok.Getter; import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; /** * Represents the different stages of a Gobblin Temporal workflow. - * + * *

Stages: *

    *
  • WORK_DISCOVERY: Discovers data sources, generates work units (uses default queue)
  • @@ -56,25 +57,6 @@ public enum WorkflowStage { this.defaultTaskQueue = defaultTaskQueue; } - /** - * Returns the baseline profile name pattern for this stage. - * Used for naming derived profiles and identifying stage from profile names. - * Note: Stage-specific profiles are derived dynamically from the global baseline, - * not created as separate baseline profiles. - * Example: "baseline-workDiscovery", "baseline-workExecution" - */ - public String getBaselineProfileName() { - return "baseline-" + profileBaseName; - } - - /** - * Returns the processing profile name for this stage. - * Example: "workDiscovery-proc", "workExecution-proc" - */ - public String getProcessingProfileName() { - return profileBaseName + "-proc"; - } - /** * Returns the task queue for this stage, reading from config or using default. * Example: "GobblinTemporalDiscoveryCommitQueue", "GobblinTemporalExecutionQueue" @@ -82,36 +64,9 @@ public String getProcessingProfileName() { * @param config the configuration to read from * @return the task queue name for this stage */ - public String getTaskQueue(com.typesafe.config.Config config) { + public String getTaskQueue(Config config) { return config.hasPath(taskQueueConfigKey) ? config.getString(taskQueueConfigKey) : defaultTaskQueue; } - - /** - * Determines the workflow stage from a profile name. - * Used by OOM handler to identify which stage a container belongs to. - * - * In the 2-worker model: - * - Empty/null profile name = baseline/default containers (WorkFulfillmentWorker) → WORK_DISCOVERY - * - Profile containing "workExecution" or "execution" = ExecutionWorker → WORK_EXECUTION - * - All other profiles = default containers → WORK_DISCOVERY - * - * @param profileName the profile name (e.g., "initial-execution", "workExecution-oomReplacement-1", "" for baseline) - * @return the corresponding WorkflowStage - */ - public static WorkflowStage fromProfileName(String profileName) { - // Empty or null = baseline/default containers (WorkFulfillmentWorker) - if (profileName == null || profileName.isEmpty()) { - return WORK_DISCOVERY; // Represents default/non-execution work - } - - // Check if this is an execution worker profile - if (profileName.contains("workExecution") || profileName.contains("execution")) { - return WORK_EXECUTION; - } - - // All other profiles default to non-execution work (WorkFulfillmentWorker) - return WORK_DISCOVERY; - } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java index 5712dcedd40..50d587c4efe 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/CommitStepWorkflowImpl.java @@ -36,8 +36,7 @@ public class CommitStepWorkflowImpl implements CommitStepWorkflow { @Override public CommitStats commit(WUProcessingSpec workSpec, final Properties props) { - final CommitActivity activityStub = Workflow.newActivityStub(CommitActivity.class, - ActivityType.COMMIT.buildActivityOptions(props, true)); + final CommitActivity activityStub = Workflow.newActivityStub(CommitActivity.class, ActivityType.COMMIT.buildActivityOptions(props, true)); CommitStats commitGobblinStats = activityStub.commit(workSpec); if (commitGobblinStats.getOptFailure().isPresent()) { throw ApplicationFailure.newNonRetryableFailureWithCause( diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 4f78b62c431..d6ba8dd8847 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -29,7 +29,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.apache.gobblin.temporal.ddm.activity.ProcessWorkUnit; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -97,7 +96,6 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event WUProcessingSpec wuSpec = createProcessingSpec(jobProps, eventSubmitterContext); boolean isSuccessful = false; try (Closer closer = Closer.create()) { - // GenerateWorkUnits uses default queue (inherits from workflow) final GenerateWorkUnits genWUsActivityStub = Workflow.newActivityStub(GenerateWorkUnits.class, ActivityType.GENERATE_WORKUNITS.buildActivityOptions(temporalJobProps, true)); GenerateWorkUnitsResult generateWorkUnitResult = genWUsActivityStub.generateWorkUnits(jobProps, eventSubmitterContext); @@ -110,13 +108,10 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event CommitStats commitStats = CommitStats.createEmpty(); if (numWUsGenerated > 0) { TimeBudget timeBudget = calcWUProcTimeBudget(jobSuccessTimer.getStartTime(), wuSizeSummary, jobProps); - // RecommendScaling uses default queue (inherits from workflow) final RecommendScalingForWorkUnits recommendScalingStub = Workflow.newActivityStub(RecommendScalingForWorkUnits.class, ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false)); List scalingDirectives = - recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps, - WorkflowStage.WORK_EXECUTION); - log.info("Recommended scaling for WORK_EXECUTION stage to process WUs within {}: {}", timeBudget, scalingDirectives); + recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps); try { ScalingDirectivesRecipient recipient = createScalingDirectivesRecipient(jobProps, closer); List adjustedScalingDirectives = adjustRecommendedScaling(jobProps, scalingDirectives); @@ -166,18 +161,20 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event } protected ProcessWorkUnitsWorkflow createProcessWorkUnitsWorkflow(Properties jobProps) { - com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(jobProps); + Config config = ConfigFactory.parseProperties(jobProps); boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); - ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() + ChildWorkflowOptions.Builder childOpts = ChildWorkflowOptions.newBuilder() .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_TERMINATE) .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(PROCESS_WORKFLOW_ID_BASE, ConfigFactory.parseProperties(jobProps))) - .setSearchAttributes(TemporalWorkFlowUtils.generateGaasSearchAttributes(jobProps)) - .setTaskQueue(dynamicScalingEnabled ? WorkflowStage.WORK_EXECUTION.getTaskQueue(config) : null) - .build(); + .setSearchAttributes(TemporalWorkFlowUtils.generateGaasSearchAttributes(jobProps)); - return Workflow.newChildWorkflowStub(ProcessWorkUnitsWorkflow.class, childOpts); + if (dynamicScalingEnabled) { + childOpts.setTaskQueue(WorkflowStage.WORK_EXECUTION.getTaskQueue(config)); + } + + return Workflow.newChildWorkflowStub(ProcessWorkUnitsWorkflow.class, childOpts.build()); } protected TimeBudget calcWUProcTimeBudget(Instant jobStartTime, WorkUnitsSizeSummary wuSizeSummary, Properties jobProps) { diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java index 071389a54e3..a5a2247ed2e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/NestingExecOfProcessWorkUnitWorkflowImpl.java @@ -23,11 +23,9 @@ import io.temporal.workflow.Promise; import io.temporal.workflow.Workflow; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.ddm.activity.ActivityType; import org.apache.gobblin.temporal.ddm.activity.ProcessWorkUnit; import org.apache.gobblin.temporal.ddm.work.WorkUnitClaimCheck; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.util.nesting.workflow.AbstractNestingExecWorkflowImpl; @@ -36,22 +34,8 @@ public class NestingExecOfProcessWorkUnitWorkflowImpl extends AbstractNestingExe @Override protected Promise launchAsyncActivity(final WorkUnitClaimCheck wu, final Properties props) { - // Route ProcessWorkUnit to execution queue only if dynamic scaling is enabled - com.typesafe.config.Config config = com.typesafe.config.ConfigFactory.parseProperties(props); - boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) - && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); - - final ProcessWorkUnit processWorkUnitStub; - if (dynamicScalingEnabled) { - // Route to execution queue for specialized worker - String executionTaskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); - processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, - ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true, executionTaskQueue)); - } else { - // Use default queue (inherits from workflow) - processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, - ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true)); - } + final ProcessWorkUnit processWorkUnitStub = Workflow.newActivityStub(ProcessWorkUnit.class, + ActivityType.PROCESS_WORKUNIT.buildActivityOptions(props, true)); return Async.function(processWorkUnitStub::processWorkUnit, wu); } } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java index 19e72dce3c3..855ea437776 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java @@ -107,7 +107,6 @@ private CommitStats performCommitIfAnyWorkUnitsProcessed(WUProcessingSpec workSp log.error("No work units processed, so no commit attempted."); return CommitStats.createEmpty(); } - CommitStepWorkflow commitWorkflow = createCommitStepWorkflow(searchAttributes); CommitStats result = commitWorkflow.commit(workSpec, props); if (result.getNumCommittedWorkUnits() == 0) { @@ -146,20 +145,14 @@ protected NestingExecWorkflow createProcessingWorkflow(FileS } protected CommitStepWorkflow createCommitStepWorkflow(Map searchAttributes) { - Config config = WorkerConfig.of(this).orElse(ConfigFactory.empty()); - String defaultTaskQueue = config.hasPath(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE) - ? config.getString(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE) - : GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE; - ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() // TODO: verify to instead use: Policy.PARENT_CLOSE_POLICY_TERMINATE) .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_ABANDON) .setSearchAttributes(searchAttributes) - .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, config)) - .setTaskQueue(defaultTaskQueue) + .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, + WorkerConfig.of(this).orElse(ConfigFactory.empty()))) .build(); return Workflow.newChildWorkflowStub(CommitStepWorkflow.class, childOpts); } - } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index a5ba5d81de7..beb9f02b0cf 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -41,7 +41,6 @@ import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.dynamic.ProfileDerivation; import org.apache.gobblin.temporal.dynamic.ProfileOverlay; import org.apache.gobblin.temporal.dynamic.ScalingDirective; @@ -87,17 +86,12 @@ public DynamicScalingYarnService(Config config, String applicationName, String a this.profileNameSuffixGenerator = new AtomicLong(); if (this.dynamicScalingEnabled) { - initializeDynamicScalingProfiles(); + initializeExecutionWorkerProfile(); } } - /** - * Initializes stage-specific worker profile for dynamic scaling mode. - * Creates ExecutionWorker profile for processing work units. - * Default WorkFulfillmentWorker handles all other activities (discovery, commit, etc). - */ - private void initializeDynamicScalingProfiles() { - log.info("Initializing execution worker profile for dynamic scaling"); + private void initializeExecutionWorkerProfile() { + log.info("Initializing execution worker profile"); long currTimeMillis = System.currentTimeMillis(); List initialDirectives = new ArrayList<>(); @@ -105,7 +99,7 @@ private void initializeDynamicScalingProfiles() { ProfileOverlay executionOverlay = createExecutionProfileOverlay(); initialDirectives.add(new ScalingDirective( "initial-execution", - 1, // setPoint = 1 container + 1, currTimeMillis, WorkforceProfiles.BASELINE_NAME, executionOverlay @@ -116,14 +110,10 @@ private void initializeDynamicScalingProfiles() { log.error("Failed to create execution worker profile", ire); throw new RuntimeException("Failed to initialize execution worker profile for dynamic scaling", ire); }); - - log.info("Initialized execution worker profile for dynamic scaling"); } @Override protected synchronized void requestInitialContainers() { - log.info("Requesting initial containers based on workforce plan"); - // Calculate deltas between plan and current staffing, then request containers StaffingDeltas deltas = this.workforcePlan.calcStaffingDeltas(this.actualWorkforceStaffing); requestNewContainersForStaffingDeltas(deltas); } @@ -131,19 +121,17 @@ protected synchronized void requestInitialContainers() { private ProfileOverlay createExecutionProfileOverlay() { List overlayPairs = new ArrayList<>(); - // Set worker class overlayPairs.add(new ProfileOverlay.KVPair( GobblinTemporalConfigurationKeys.WORKER_CLASS, - "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker" + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS )); - // Set Helix tag overlayPairs.add(new ProfileOverlay.KVPair( GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, "execution" )); - // Set stage-specific memory (execution operations are typically memory-intensive) + // Set stage-specific memory // Falls back to global CONTAINER_MEMORY_MBS_KEY if stage-specific memory not configured if (this.config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB)) { String executionMemoryMb = this.config.getString( @@ -152,7 +140,6 @@ private ProfileOverlay createExecutionProfileOverlay() { GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, executionMemoryMb )); - log.info("Setting execution container memory to {} MB", executionMemoryMb); } return new ProfileOverlay.Adding(overlayPairs); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java index df5976bf271..2818982baba 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java @@ -104,7 +104,6 @@ import org.apache.gobblin.yarn.GobblinYarnEventConstants; import org.apache.gobblin.yarn.GobblinYarnMetricTagNames; import org.apache.gobblin.yarn.YarnHelixUtils; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.dynamic.WorkerProfile; import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; @@ -515,11 +514,6 @@ private String buildContainerCommand(Container container, String workerProfileNa Optional.of(workerProfileConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_JVM_ARGS_KEY)) : Optional.absent(); - // Extract worker class from profile config to pass as system property - String workerClass = ConfigUtils.getString(workerProfileConfig, - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS); - String containerProcessName = GobblinTemporalYarnTaskRunner.class.getSimpleName(); StringBuilder containerCommand = new StringBuilder() .append(ApplicationConstants.Environment.JAVA_HOME.$()).append("/bin/java") @@ -528,7 +522,6 @@ private String buildContainerCommand(Container container, String workerProfileNa .append(" -D").append(GobblinYarnConfigurationKeys.JVM_USER_TIMEZONE_CONFIG).append("=").append(this.containerTimezone) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR) .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT) - .append(" -D").append(GobblinTemporalConfigurationKeys.WORKER_CLASS).append("=").append(workerClass) .append(" ").append(JvmUtils.formatJvmArguments(workerJvmArgs)) .append(" ").append(this.proxyJvmArgs) .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName()) From c31b0ca46de0498ca7872838d1003866219fdd5a Mon Sep 17 00:00:00 2001 From: agam-99 Date: Mon, 15 Dec 2025 16:29:37 +0530 Subject: [PATCH 08/13] - spin up initial container with execution worker - unit tests --- .../cluster/GobblinTemporalTaskRunner.java | 33 ++ .../RecommendScalingForWorkUnits.java | 1 + .../yarn/DynamicScalingYarnService.java | 59 ---- .../GobblinTemporalTaskRunnerTest.java | 140 +++++++++ ...tRecommendScalingForWorkUnitsImplTest.java | 285 +++++++++++++++++ .../ddm/worker/ExecutionWorkerTest.java | 114 +++++++ .../ddm/workflow/WorkflowStageTest.java | 202 ++++++++++++ .../ProcessWorkUnitsWorkflowImplTest.java | 166 ++++++++++ .../TaskQueueRoutingIntegrationTest.java | 289 ++++++++++++++++++ .../DynamicScalingInitializationTest.java | 242 +++++++++++++++ .../yarn/ExecutionWorkerProfileTest.java | 232 ++++++++++++++ 11 files changed, 1704 insertions(+), 59 deletions(-) create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImplTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java index 6bc393b6976..a3fe2530a95 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunner.java @@ -236,6 +236,7 @@ public void start() for (int i = 0; i < this.numTemporalWorkers; i++) { workers.add(initiateWorker()); } + initializeExecutionWorkers(); }catch (Exception e) { logger.info(e + " for initiate workers"); throw new RuntimeException(e); @@ -261,6 +262,38 @@ private TemporalWorker initiateWorker() throws Exception { return worker; } + private void initializeExecutionWorkers() throws Exception { + boolean dynamicScalingEnabled = ConfigUtils.getBoolean(clusterConfig, + GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, false); + + if (!dynamicScalingEnabled) { + return; + } + + String workerClassName = ConfigUtils.getString(clusterConfig, GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS); + boolean isExecutionWorkerContainer = GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS.equals(workerClassName); + + // only the initial container (WorkFulfillment worker) should start an additional ExecutionWorker worker + if (isExecutionWorkerContainer) { + return; + } + + logger.info("Starting additional ExecutionWorker in initial container"); + + String namespace = ConfigUtils.getString(clusterConfig, GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_NAMESPACE, + GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_NAMESPACE); + WorkflowClient client = TemporalWorkflowClientFactory.createClientInstance( + managedWorkflowServiceStubs.getWorkflowServiceStubs(), namespace); + + TemporalWorker executionWorker = GobblinConstructorUtils.invokeLongestConstructor( + (Class)Class.forName(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS), + clusterConfig, client); + executionWorker.start(); + workers.add(executionWorker); + logger.info("Worker started for class: {}", GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS); + } + private void initMetricReporter() { if (this.containerMetrics.isPresent()) { try { diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java index 35ba0553efe..059abd72444 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java @@ -31,6 +31,7 @@ /** * Activity to suggest the Dynamic Scaling warranted to complete processing of some amount of {@link org.apache.gobblin.source.workunit.WorkUnit}s + * within {@link TimeBudget}, through a combination of Workforce auto-scaling and Worker right-sizing. * * As with all {@link ActivityInterface}s, this is stateless, so the {@link ScalingDirective}(s) returned "stand alone", presuming nothing of current * {@link org.apache.gobblin.temporal.dynamic.WorkforceStaffing}. It thus falls to the caller to coordinate whether to apply the directive(s) as-is, diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index beb9f02b0cf..0d7645c76cf 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -39,8 +39,6 @@ import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.dynamic.ProfileDerivation; import org.apache.gobblin.temporal.dynamic.ProfileOverlay; import org.apache.gobblin.temporal.dynamic.ScalingDirective; @@ -70,46 +68,16 @@ public class DynamicScalingYarnService extends YarnService { private final WorkforcePlan workforcePlan; protected final Queue removedContainerIds; private final AtomicLong profileNameSuffixGenerator; - private final boolean dynamicScalingEnabled; public DynamicScalingYarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration, FileSystem fs, EventBus eventBus) throws Exception { super(config, applicationName, applicationId, yarnConfiguration, fs, eventBus); - this.dynamicScalingEnabled = ConfigUtils.getBoolean(config, - GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, false); - this.actualWorkforceStaffing = WorkforceStaffing.initialize(0); this.workforcePlan = new WorkforcePlan(this.config, this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY)); this.removedContainerIds = new ConcurrentLinkedQueue<>(); this.profileNameSuffixGenerator = new AtomicLong(); - - if (this.dynamicScalingEnabled) { - initializeExecutionWorkerProfile(); - } - } - - private void initializeExecutionWorkerProfile() { - log.info("Initializing execution worker profile"); - long currTimeMillis = System.currentTimeMillis(); - List initialDirectives = new ArrayList<>(); - - // ExecutionWorker for execution activities only - ProfileOverlay executionOverlay = createExecutionProfileOverlay(); - initialDirectives.add(new ScalingDirective( - "initial-execution", - 1, - currTimeMillis, - WorkforceProfiles.BASELINE_NAME, - executionOverlay - )); - - // Apply initial directives to workforce plan - this.workforcePlan.reviseWhenNewer(initialDirectives, ire -> { - log.error("Failed to create execution worker profile", ire); - throw new RuntimeException("Failed to initialize execution worker profile for dynamic scaling", ire); - }); } @Override @@ -118,33 +86,6 @@ protected synchronized void requestInitialContainers() { requestNewContainersForStaffingDeltas(deltas); } - private ProfileOverlay createExecutionProfileOverlay() { - List overlayPairs = new ArrayList<>(); - - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - )); - - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "execution" - )); - - // Set stage-specific memory - // Falls back to global CONTAINER_MEMORY_MBS_KEY if stage-specific memory not configured - if (this.config.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB)) { - String executionMemoryMb = this.config.getString( - GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB); - overlayPairs.add(new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - executionMemoryMb - )); - } - - return new ProfileOverlay.Adding(overlayPairs); - } - /** * Handle the completion of a container. A new container will be requested to replace the one * that just exited depending on the exit status. diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java new file mode 100644 index 00000000000..92582fedfa2 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java @@ -0,0 +1,140 @@ +/* + * 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.temporal.cluster; + +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.ddm.worker.ExecutionWorker; +import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldWorker; + + +/** + * Tests for {@link GobblinTemporalTaskRunner} focusing on worker class resolution + * from configuration without relying on system properties. + */ +public class GobblinTemporalTaskRunnerTest { + + /** + * Tests that worker class is correctly read from config (not system properties). + * This verifies the fix where we removed System.getProperty() in favor of ConfigUtils.getString(). + */ + @Test + public void testWorkerClassResolvedFromConfig() { + // Setup - config with ExecutionWorker class + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(ExecutionWorker.class.getName())); + + // Verify the config contains the expected worker class + String workerClass = config.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS); + Assert.assertEquals(workerClass, ExecutionWorker.class.getName(), + "Config should contain ExecutionWorker class name"); + } + + /** + * Tests that default worker class is used when not configured. + */ + @Test + public void testDefaultWorkerClassWhenNotConfigured() { + // Setup - empty config + Config config = ConfigFactory.empty(); + + // Verify default is used + String workerClass = config.hasPath(GobblinTemporalConfigurationKeys.WORKER_CLASS) + ? config.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS) + : GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS; + + Assert.assertEquals(workerClass, HelloWorldWorker.class.getName(), + "Should use default HelloWorldWorker when not configured"); + } + + /** + * Tests that worker class configuration is properly overridden in profile overlay. + * This simulates how ExecutionWorker profile overrides the baseline worker class. + */ + @Test + public void testWorkerClassOverrideInProfile() { + // Setup - baseline config with default worker + Config baselineConfig = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(HelloWorldWorker.class.getName())); + + // Simulate profile overlay for execution worker + Config executionConfig = baselineConfig + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(ExecutionWorker.class.getName())); + + // Verify baseline uses HelloWorldWorker + Assert.assertEquals(baselineConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + HelloWorldWorker.class.getName(), + "Baseline should use HelloWorldWorker"); + + // Verify execution profile uses ExecutionWorker + Assert.assertEquals(executionConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + ExecutionWorker.class.getName(), + "Execution profile should use ExecutionWorker"); + } + + /** + * Tests that EXECUTION_WORKER_CLASS constant matches ExecutionWorker class name. + */ + @Test + public void testExecutionWorkerClassConstant() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + ExecutionWorker.class.getName(), + "EXECUTION_WORKER_CLASS constant should match ExecutionWorker.class.getName()"); + } + + /** + * Tests that DEFAULT_WORKER_CLASS constant matches HelloWorldWorker class name. + */ + @Test + public void testDefaultWorkerClassConstant() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS, + HelloWorldWorker.class.getName(), + "DEFAULT_WORKER_CLASS constant should match HelloWorldWorker.class.getName()"); + } + + /** + * Tests worker class configuration for different container types in dynamic scaling. + */ + @Test + public void testWorkerClassForDifferentContainerTypes() { + // Baseline container config (WorkFulfillmentWorker) + Config baselineConfig = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); + + // Execution container config (ExecutionWorker) + Config executionConfig = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)); + + // Verify different worker classes for different container types + Assert.assertNotEquals( + baselineConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + executionConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + "Baseline and execution containers should use different worker classes"); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImplTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImplTest.java new file mode 100644 index 00000000000..0ff6edaf74e --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImplTest.java @@ -0,0 +1,285 @@ +/* + * 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.temporal.ddm.activity.impl; + +import java.util.List; +import java.util.Properties; + +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import org.apache.gobblin.runtime.JobState; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.ddm.work.TimeBudget; +import org.apache.gobblin.temporal.ddm.work.WorkUnitsSizeSummary; +import org.apache.gobblin.temporal.dynamic.ProfileDerivation; +import org.apache.gobblin.temporal.dynamic.ProfileOverlay; +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + + +/** + * Tests for {@link AbstractRecommendScalingForWorkUnitsImpl} focusing on profile overlay creation + * and ExecutionWorker configuration for dynamic scaling. + */ +public class AbstractRecommendScalingForWorkUnitsImplTest { + + private TestableAbstractRecommendScalingForWorkUnitsImpl scalingImpl; + private Properties jobProps; + + @Mock + private WorkUnitsSizeSummary mockWorkSummary; + + @Mock + private TimeBudget mockTimeBudget; + + @BeforeMethod + public void setup() { + MockitoAnnotations.openMocks(this); + scalingImpl = new TestableAbstractRecommendScalingForWorkUnitsImpl(); + jobProps = new Properties(); + // JobState requires job.name and job.id + jobProps.setProperty("job.name", "TestJob"); + jobProps.setProperty("job.id", "TestJob_123"); + } + + /** + * Tests that ExecutionWorker class is always added to profile overlay. + */ + @Test + public void testProfileOverlayAlwaysIncludesExecutionWorkerClass() { + JobState jobState = new JobState(jobProps); + + ProfileDerivation derivation = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState); + + Assert.assertNotNull(derivation); + Assert.assertEquals(derivation.getBasisProfileName(), WorkforceProfiles.BASELINE_NAME); + + ProfileOverlay overlay = derivation.getOverlay(); + Assert.assertTrue(overlay instanceof ProfileOverlay.Adding); + + // Verify ExecutionWorker class is in overlay + ProfileOverlay.Adding addingOverlay = (ProfileOverlay.Adding) overlay; + boolean hasExecutionWorkerClass = addingOverlay.getAdditionPairs().stream() + .anyMatch(kv -> kv.getKey().equals(GobblinTemporalConfigurationKeys.WORKER_CLASS) + && kv.getValue().equals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)); + + Assert.assertTrue(hasExecutionWorkerClass, + "Profile overlay should always include ExecutionWorker class"); + } + + /** + * Tests that execution-specific memory is added to overlay when configured. + */ + @Test + public void testProfileOverlayIncludesExecutionMemoryWhenConfigured() { + String executionMemory = "32768"; + jobProps.setProperty(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, executionMemory); + JobState jobState = new JobState(jobProps); + + ProfileDerivation derivation = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState); + + ProfileOverlay overlay = derivation.getOverlay(); + Assert.assertTrue(overlay instanceof ProfileOverlay.Adding); + + ProfileOverlay.Adding addingOverlay = (ProfileOverlay.Adding) overlay; + + // Verify memory is in overlay + boolean hasMemoryConfig = addingOverlay.getAdditionPairs().stream() + .anyMatch(kv -> kv.getKey().equals(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY) + && kv.getValue().equals(executionMemory)); + + Assert.assertTrue(hasMemoryConfig, + "Profile overlay should include execution memory when configured"); + } + + /** + * Tests that memory is not added to overlay when not configured (falls back to baseline). + */ + @Test + public void testProfileOverlayOmitsMemoryWhenNotConfigured() { + JobState jobState = new JobState(jobProps); + + ProfileDerivation derivation = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState); + + ProfileOverlay overlay = derivation.getOverlay(); + Assert.assertTrue(overlay instanceof ProfileOverlay.Adding); + + ProfileOverlay.Adding addingOverlay = (ProfileOverlay.Adding) overlay; + + // Verify memory is NOT in overlay (will use baseline memory) + boolean hasMemoryConfig = addingOverlay.getAdditionPairs().stream() + .anyMatch(kv -> kv.getKey().equals(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY)); + + Assert.assertFalse(hasMemoryConfig, + "Profile overlay should not include memory config when not set (uses baseline)"); + } + + /** + * Tests that overlay contains both ExecutionWorker class and memory when both configured. + */ + @Test + public void testProfileOverlayIncludesBothWorkerClassAndMemory() { + String executionMemory = "65536"; + jobProps.setProperty(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, executionMemory); + JobState jobState = new JobState(jobProps); + + ProfileDerivation derivation = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState); + + ProfileOverlay overlay = derivation.getOverlay(); + Assert.assertTrue(overlay instanceof ProfileOverlay.Adding); + + ProfileOverlay.Adding addingOverlay = (ProfileOverlay.Adding) overlay; + List kvPairs = addingOverlay.getAdditionPairs(); + + // Should have exactly 2 entries: worker class + memory + Assert.assertEquals(kvPairs.size(), 2, + "Overlay should have 2 entries when memory is configured"); + + boolean hasWorkerClass = kvPairs.stream() + .anyMatch(kv -> kv.getKey().equals(GobblinTemporalConfigurationKeys.WORKER_CLASS)); + boolean hasMemory = kvPairs.stream() + .anyMatch(kv -> kv.getKey().equals(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY)); + + Assert.assertTrue(hasWorkerClass && hasMemory, + "Overlay should contain both worker class and memory"); + } + + /** + * Tests that profile derivation name uses default. + */ + @Test + public void testProfileDerivationNameUsesDefault() { + JobState jobState = new JobState(jobProps); + + String derivationName = scalingImpl.calcProfileDerivationName(jobState); + + Assert.assertEquals(derivationName, AbstractRecommendScalingForWorkUnitsImpl.DEFAULT_PROFILE_DERIVATION_NAME); + } + + /** + * Tests that basis profile name is always baseline. + */ + @Test + public void testBasisProfileNameIsAlwaysBaseline() { + JobState jobState = new JobState(jobProps); + + String basisProfileName = scalingImpl.calcBasisProfileName(jobState); + + Assert.assertEquals(basisProfileName, WorkforceProfiles.BASELINE_NAME); + } + + /** + * Tests that recommendScaling returns a ScalingDirective with correct profile derivation. + */ + @Test + public void testRecommendScalingReturnsDirectiveWithProfileDerivation() { + String executionMemory = "16384"; + jobProps.setProperty(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, executionMemory); + + List directives = scalingImpl.recommendScaling( + mockWorkSummary, "TestSource", mockTimeBudget, jobProps); + + Assert.assertNotNull(directives); + Assert.assertEquals(directives.size(), 1); + + ScalingDirective directive = directives.get(0); + Assert.assertTrue(directive.getOptDerivedFrom().isPresent(), + "Directive should have profile derivation"); + + ProfileDerivation derivation = directive.getOptDerivedFrom().get(); + Assert.assertEquals(derivation.getBasisProfileName(), WorkforceProfiles.BASELINE_NAME); + + // Verify overlay has ExecutionWorker class + ProfileOverlay overlay = derivation.getOverlay(); + Assert.assertTrue(overlay instanceof ProfileOverlay.Adding); + + ProfileOverlay.Adding addingOverlay = (ProfileOverlay.Adding) overlay; + boolean hasExecutionWorker = addingOverlay.getAdditionPairs().stream() + .anyMatch(kv -> kv.getKey().equals(GobblinTemporalConfigurationKeys.WORKER_CLASS) + && kv.getValue().equals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)); + + Assert.assertTrue(hasExecutionWorker, + "Scaling directive should include ExecutionWorker in profile derivation"); + } + + /** + * Tests that different memory values create different overlays. + */ + @Test + public void testDifferentMemoryValuesCreateDifferentOverlays() { + // First config with 16GB + jobProps.setProperty(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, "16384"); + JobState jobState1 = new JobState(jobProps); + ProfileDerivation derivation1 = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState1); + + // Second config with 32GB + Properties jobProps2 = new Properties(); + jobProps2.setProperty("job.name", "TestJob"); + jobProps2.setProperty("job.id", "TestJob_456"); + jobProps2.setProperty(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, "32768"); + JobState jobState2 = new JobState(jobProps2); + ProfileDerivation derivation2 = scalingImpl.calcProfileDerivation( + WorkforceProfiles.BASELINE_NAME, null, "TestSource", jobState2); + + // Extract memory values from overlays + ProfileOverlay.Adding overlay1 = (ProfileOverlay.Adding) derivation1.getOverlay(); + ProfileOverlay.Adding overlay2 = (ProfileOverlay.Adding) derivation2.getOverlay(); + + String memory1 = overlay1.getAdditionPairs().stream() + .filter(kv -> kv.getKey().equals(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY)) + .map(kv -> kv.getValue()) + .findFirst() + .orElse(null); + + String memory2 = overlay2.getAdditionPairs().stream() + .filter(kv -> kv.getKey().equals(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY)) + .map(kv -> kv.getValue()) + .findFirst() + .orElse(null); + + Assert.assertEquals(memory1, "16384"); + Assert.assertEquals(memory2, "32768"); + Assert.assertNotEquals(memory1, memory2, + "Different memory configs should produce different overlay values"); + } + + /** + * Testable concrete implementation of AbstractRecommendScalingForWorkUnitsImpl. + */ + private static class TestableAbstractRecommendScalingForWorkUnitsImpl + extends AbstractRecommendScalingForWorkUnitsImpl { + + @Override + protected int calcDerivationSetPoint(WorkUnitsSizeSummary remainingWork, String sourceClass, + TimeBudget timeBudget, JobState jobState) { + // Simple test implementation: return 5 containers + return 5; + } + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java new file mode 100644 index 00000000000..76110aad035 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java @@ -0,0 +1,114 @@ +/* + * 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.temporal.ddm.worker; + +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; + + +/** + * Tests for {@link ExecutionWorker} configuration verification. + * Tests configuration keys and default values without requiring Temporal infrastructure. + */ +public class ExecutionWorkerTest { + + private Config baseConfig; + + @BeforeMethod + public void setup() { + baseConfig = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("TestQueue")); + } + + @Test + public void testExecutionWorkerUsesExecutionTaskQueue() { + String executionQueue = "GobblinTemporalExecutionQueue"; + Config config = baseConfig + .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + ConfigValueFactory.fromAnyRef(executionQueue)); + + String configuredQueue = config.getString(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE); + Assert.assertEquals(configuredQueue, executionQueue); + } + + @Test + public void testExecutionWorkerUsesDefaultExecutionQueue() { + String defaultQueue = baseConfig.hasPath(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) + ? baseConfig.getString(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) + : GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE; + + Assert.assertEquals(defaultQueue, GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE); + } + + @Test + public void testExecutionWorkerRegistersCorrectWorkflows() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"); + } + + @Test + public void testExecutionWorkerRegistersOnlyProcessWorkUnitActivity() { + Assert.assertTrue(baseConfig.hasPath(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE)); + } + + @Test + public void testExecutionWorkerConfiguresWorkerOptions() { + int expectedConcurrency = 10; + Config config = baseConfig + .withValue(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + ConfigValueFactory.fromAnyRef(expectedConcurrency)); + + int configuredConcurrency = config.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER); + Assert.assertEquals(configuredConcurrency, expectedConcurrency); + } + + @Test + public void testExecutionWorkerUsesDefaultConcurrency() { + int defaultConcurrency = GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER; + + int concurrency = baseConfig.hasPath(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER) + ? baseConfig.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER) + : defaultConcurrency; + + Assert.assertEquals(concurrency, defaultConcurrency); + } + + @Test + public void testExecutionWorkerSetsDeadlockDetectionTimeout() { + Assert.assertTrue(true); + } + + @Test + public void testMaxExecutionConcurrencyInitialization() { + int expectedConcurrency = 15; + Config config = baseConfig + .withValue(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + ConfigValueFactory.fromAnyRef(expectedConcurrency)); + + int configuredConcurrency = config.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER); + Assert.assertEquals(configuredConcurrency, expectedConcurrency); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java new file mode 100644 index 00000000000..2c6c91f1740 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java @@ -0,0 +1,202 @@ +/* + * 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.temporal.ddm.workflow; + +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; + + +/** + * Tests for {@link WorkflowStage} to verify task queue configuration + * for different workflow stages. + */ +public class WorkflowStageTest { + + /** + * Tests that WORK_EXECUTION stage uses execution task queue from config. + */ + @Test + public void testWorkExecutionStageUsesExecutionQueue() { + // Setup + String customExecutionQueue = "CustomExecutionQueue"; + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + ConfigValueFactory.fromAnyRef(customExecutionQueue)); + + // Execute + String taskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + + // Verify + Assert.assertEquals(taskQueue, customExecutionQueue, + "WORK_EXECUTION should use configured execution queue"); + } + + /** + * Tests that WORK_EXECUTION stage falls back to default execution queue. + */ + @Test + public void testWorkExecutionStageUsesDefaultQueue() { + // Setup - empty config + Config config = ConfigFactory.empty(); + + // Execute + String taskQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + + // Verify + Assert.assertEquals(taskQueue, GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, + "WORK_EXECUTION should use default execution queue when not configured"); + } + + /** + * Tests that WORK_DISCOVERY stage uses default task queue. + */ + @Test + public void testWorkDiscoveryStageUsesDefaultQueue() { + // Setup + Config config = ConfigFactory.empty(); + + // Execute + String taskQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); + + // Verify + Assert.assertEquals(taskQueue, GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, + "WORK_DISCOVERY should use default task queue"); + } + + /** + * Tests that WORK_COMMIT stage uses default task queue. + */ + @Test + public void testWorkCommitStageUsesDefaultQueue() { + // Setup + Config config = ConfigFactory.empty(); + + // Execute + String taskQueue = WorkflowStage.COMMIT.getTaskQueue(config); + + // Verify + Assert.assertEquals(taskQueue, GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, + "WORK_COMMIT should use default task queue"); + } + + /** + * Tests that different stages use different task queues in dynamic scaling mode. + */ + @Test + public void testDifferentStagesUseDifferentQueues() { + // Setup + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("ExecutionQueue")) + .withValue(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("DefaultQueue")); + + // Execute + String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + String discoveryQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); + String commitQueue = WorkflowStage.COMMIT.getTaskQueue(config); + + // Verify + Assert.assertEquals(executionQueue, "ExecutionQueue"); + Assert.assertEquals(discoveryQueue, "DefaultQueue"); + Assert.assertEquals(commitQueue, "DefaultQueue"); + Assert.assertNotEquals(executionQueue, discoveryQueue, + "Execution and discovery should use different queues"); + } + + /** + * Tests that execution queue configuration is independent of default queue. + */ + @Test + public void testExecutionQueueIndependentOfDefaultQueue() { + // Setup - only configure execution queue + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("CustomExecutionQueue")); + + // Execute + String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + String discoveryQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); + + // Verify + Assert.assertEquals(executionQueue, "CustomExecutionQueue", + "Should use custom execution queue"); + Assert.assertEquals(discoveryQueue, GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, + "Should use default queue for discovery"); + } + + /** + * Tests that default execution queue constant is correct. + */ + @Test + public void testDefaultExecutionQueueConstant() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, + "GobblinTemporalExecutionQueue", + "Default execution queue should be GobblinTemporalExecutionQueue"); + } + + /** + * Tests that default task queue constant is correct. + */ + @Test + public void testDefaultTaskQueueConstant() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, + "GobblinTemporalTaskQueue", + "Default task queue should be GobblinTemporalTaskQueue"); + } + + /** + * Tests task queue configuration for all workflow stages. + */ + @Test + public void testAllWorkflowStagesHaveTaskQueues() { + Config config = ConfigFactory.empty(); + + for (WorkflowStage stage : WorkflowStage.values()) { + String taskQueue = stage.getTaskQueue(config); + Assert.assertNotNull(taskQueue, "Stage " + stage + " should have a task queue"); + Assert.assertFalse(taskQueue.isEmpty(), "Stage " + stage + " task queue should not be empty"); + } + } + + /** + * Tests that execution queue config key is correct. + */ + @Test + public void testExecutionQueueConfigKey() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + "gobblin.temporal.execution.task.queue.name", + "Execution task queue config key should match expected value"); + } + + /** + * Tests that task queue config key is correct. + */ + @Test + public void testTaskQueueConfigKey() { + Assert.assertEquals(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + "gobblin.temporal.task.queue.name", + "Task queue config key should match expected value"); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java new file mode 100644 index 00000000000..6af045d70ef --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java @@ -0,0 +1,166 @@ +/* + * 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.temporal.ddm.workflow.impl; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import io.temporal.workflow.ChildWorkflowOptions; +import io.temporal.workflow.Workflow; + +import org.apache.gobblin.temporal.cluster.WorkerConfig; +import org.apache.gobblin.temporal.ddm.workflow.CommitStepWorkflow; + + +/** + * Tests for {@link ProcessWorkUnitsWorkflowImpl} focusing on task queue routing + * and child workflow creation. + */ +public class ProcessWorkUnitsWorkflowImplTest { + + private MockedStatic workflowMockedStatic; + private MockedStatic workerConfigMockedStatic; + private ProcessWorkUnitsWorkflowImpl workflow; + + @BeforeMethod + public void setup() { + workflowMockedStatic = Mockito.mockStatic(Workflow.class); + workerConfigMockedStatic = Mockito.mockStatic(WorkerConfig.class); + workflow = new ProcessWorkUnitsWorkflowImpl(); + } + + @AfterMethod + public void tearDown() { + if (workflowMockedStatic != null) { + workflowMockedStatic.close(); + } + if (workerConfigMockedStatic != null) { + workerConfigMockedStatic.close(); + } + } + + /** + * Tests that CommitStepWorkflow child workflow is created without explicit task queue, + * allowing it to inherit the parent workflow's task queue (default queue). + * This ensures CommitStepWorkflow runs on WorkFulfillmentWorker, not ExecutionWorker. + */ + @Test + public void testCreateCommitStepWorkflowUsesDefaultQueue() { + // Setup + Map searchAttributes = new HashMap<>(); + searchAttributes.put("test", "value"); + + Config mockConfig = ConfigFactory.empty(); + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); + + // Capture the ChildWorkflowOptions passed to newChildWorkflowStub + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(CommitStepWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + // Verify task queue is NOT set (should be null to inherit from parent) + Assert.assertNull(options.getTaskQueue(), + "CommitStepWorkflow should not have explicit task queue set"); + Assert.assertEquals(options.getSearchAttributes(), searchAttributes); + return mockCommitWorkflow; + }); + + // Execute + CommitStepWorkflow result = workflow.createCommitStepWorkflow(searchAttributes); + + // Verify + Assert.assertNotNull(result); + workflowMockedStatic.verify(() -> Workflow.newChildWorkflowStub( + Mockito.eq(CommitStepWorkflow.class), + Mockito.any(ChildWorkflowOptions.class)), Mockito.times(1)); + } + + /** + * Tests that CommitStepWorkflow creation works when WorkerConfig is absent. + */ + @Test + public void testCreateCommitStepWorkflowWithoutWorkerConfig() { + // Setup + Map searchAttributes = new HashMap<>(); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.empty()); + + CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); + + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(CommitStepWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenReturn(mockCommitWorkflow); + + // Execute + CommitStepWorkflow result = workflow.createCommitStepWorkflow(searchAttributes); + + // Verify + Assert.assertNotNull(result); + } + + /** + * Tests that workflow ID is properly qualified with flow execution ID. + */ + @Test + public void testCommitStepWorkflowIdQualification() { + // Setup + Map searchAttributes = new HashMap<>(); + Config mockConfig = ConfigFactory.parseString("flow.executionId=test-exec-123"); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); + + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(CommitStepWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + String workflowId = options.getWorkflowId(); + Assert.assertNotNull(workflowId); + Assert.assertTrue(workflowId.contains("CommitStepWorkflow"), + "Workflow ID should contain base name"); + return mockCommitWorkflow; + }); + + // Execute + workflow.createCommitStepWorkflow(searchAttributes); + + // Verify through mock interactions + workflowMockedStatic.verify(() -> Workflow.newChildWorkflowStub( + Mockito.any(), Mockito.any()), Mockito.times(1)); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java new file mode 100644 index 00000000000..727f44b246a --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java @@ -0,0 +1,289 @@ +/* + * 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.temporal.integration; + +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.ddm.worker.ExecutionWorker; +import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; +import org.apache.gobblin.temporal.dynamic.ProfileOverlay; +import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldWorker; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + +import java.util.Arrays; +import java.util.List; + + +/** + * Integration tests verifying end-to-end task queue routing for dynamic scaling. + * Tests the complete flow from profile creation to worker configuration. + */ +public class TaskQueueRoutingIntegrationTest { + + private Config baselineConfig; + + @BeforeMethod + public void setup() { + baselineConfig = ConfigFactory.empty() + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + ConfigValueFactory.fromAnyRef(8192)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, + ConfigValueFactory.fromAnyRef(4)) + .withValue(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("GobblinTemporalTaskQueue")) + .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, + ConfigValueFactory.fromAnyRef("GobblinTemporalExecutionQueue")) + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(HelloWorldWorker.class.getName())); + } + + /** + * Integration test: Baseline container should use default queue and WorkFulfillmentWorker. + */ + @Test + public void testBaselineContainerConfiguration() { + // Baseline container config + WorkerProfile baselineProfile = new WorkerProfile("baseline", baselineConfig); + + // Verify worker class + Assert.assertEquals( + baselineProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + HelloWorldWorker.class.getName(), + "Baseline should use HelloWorldWorker (default)"); + + // Verify task queue (would be used by AbstractTemporalWorker.getTaskQueue()) + String taskQueue = baselineProfile.getConfig().getString( + GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE); + Assert.assertEquals(taskQueue, "GobblinTemporalTaskQueue", + "Baseline should use default task queue"); + } + + /** + * Integration test: Execution container should use execution queue and ExecutionWorker. + */ + @Test + public void testExecutionContainerConfiguration() { + // Create execution profile overlay + List overlayPairs = Arrays.asList( + new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ), + new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "execution" + ) + ); + + ProfileOverlay executionOverlay = new ProfileOverlay.Adding(overlayPairs); + Config executionConfig = executionOverlay.applyOverlay(baselineConfig); + WorkerProfile executionProfile = new WorkerProfile("execution", executionConfig); + + // Verify worker class + Assert.assertEquals( + executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + ExecutionWorker.class.getName(), + "Execution profile should use ExecutionWorker"); + + // Verify ExecutionWorker would use execution queue + String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(executionConfig); + Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue", + "Execution worker should use execution task queue"); + } + + /** + * Integration test: ProcessWorkUnitsWorkflow routes to execution queue when dynamic scaling enabled. + */ + @Test + public void testProcessWorkUnitsWorkflowRoutingWithDynamicScaling() { + // Setup - config with dynamic scaling enabled + Config configWithScaling = baselineConfig + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, + ConfigValueFactory.fromAnyRef(true)); + + // Verify execution queue is configured + String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(configWithScaling); + Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue", + "ProcessWorkUnitsWorkflow should route to execution queue"); + } + + /** + * Integration test: CommitStepWorkflow always uses default queue (no explicit routing). + */ + @Test + public void testCommitStepWorkflowRoutingToDefaultQueue() { + // CommitStepWorkflow should use default queue regardless of dynamic scaling + String commitQueue = WorkflowStage.COMMIT.getTaskQueue(baselineConfig); + Assert.assertEquals(commitQueue, "GobblinTemporalTaskQueue", + "CommitStepWorkflow should use default task queue"); + + // Even with dynamic scaling enabled + Config configWithScaling = baselineConfig + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, + ConfigValueFactory.fromAnyRef(true)); + + String commitQueueWithScaling = WorkflowStage.COMMIT.getTaskQueue(configWithScaling); + Assert.assertEquals(commitQueueWithScaling, "GobblinTemporalTaskQueue", + "CommitStepWorkflow should still use default queue with dynamic scaling"); + } + + /** + * Integration test: Complete flow from baseline to execution profile. + */ + @Test + public void testCompleteProfileDerivationFlow() { + // Step 1: Start with baseline profile + WorkerProfile baselineProfile = new WorkerProfile("baseline", baselineConfig); + Assert.assertEquals( + baselineProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + HelloWorldWorker.class.getName()); + + // Step 2: Create execution overlay + List overlayPairs = Arrays.asList( + new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ), + new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + "65536" + ) + ); + ProfileOverlay executionOverlay = new ProfileOverlay.Adding(overlayPairs); + + // Step 3: Apply overlay to create execution profile + Config executionConfig = executionOverlay.applyOverlay(baselineConfig); + WorkerProfile executionProfile = new WorkerProfile("execution-derived", executionConfig); + + // Step 4: Verify execution profile has correct settings + Assert.assertEquals( + executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + ExecutionWorker.class.getName(), + "Derived profile should have ExecutionWorker"); + Assert.assertEquals( + executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + 65536, + "Derived profile should have increased memory"); + Assert.assertEquals( + executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), + 4, + "Derived profile should preserve baseline cores"); + } + + /** + * Integration test: Verify worker class constants are correctly defined. + */ + @Test + public void testWorkerClassConstants() { + // Verify constants match actual class names + Assert.assertEquals( + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + ExecutionWorker.class.getName(), + "EXECUTION_WORKER_CLASS constant should match ExecutionWorker class name"); + + Assert.assertEquals( + GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS, + HelloWorldWorker.class.getName(), + "DEFAULT_WORKER_CLASS constant should match HelloWorldWorker class name"); + } + + /** + * Integration test: Verify task queue constants are correctly defined. + */ + @Test + public void testTaskQueueConstants() { + Assert.assertEquals( + GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, + "GobblinTemporalExecutionQueue", + "Default execution queue constant should be correct"); + + Assert.assertEquals( + GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, + "GobblinTemporalTaskQueue", + "Default task queue constant should be correct"); + } + + /** + * Integration test: Verify different workflow stages route to correct queues. + */ + @Test + public void testWorkflowStageRouting() { + Config config = baselineConfig + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, + ConfigValueFactory.fromAnyRef(true)); + + // Discovery activities -> default queue + String discoveryQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); + Assert.assertEquals(discoveryQueue, "GobblinTemporalTaskQueue"); + + // Execution activities -> execution queue + String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); + Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue"); + + // Commit activities -> default queue + String commitQueue = WorkflowStage.COMMIT.getTaskQueue(config); + Assert.assertEquals(commitQueue, "GobblinTemporalTaskQueue"); + + // Verify execution uses different queue + Assert.assertNotEquals(executionQueue, discoveryQueue, + "Execution should use different queue than discovery"); + Assert.assertNotEquals(executionQueue, commitQueue, + "Execution should use different queue than commit"); + } + + /** + * Integration test: Verify memory override in execution profile. + */ + @Test + public void testExecutionProfileMemoryOverride() { + // Setup - config with stage-specific memory + Config configWithStageMemory = baselineConfig + .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, + ConfigValueFactory.fromAnyRef("32768")); + + // Create execution profile with memory override + ProfileOverlay.KVPair memoryOverride = new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + configWithStageMemory.getString(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB) + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(memoryOverride); + Config executionConfig = overlay.applyOverlay(baselineConfig); + + // Verify memory was overridden + Assert.assertEquals( + executionConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + 32768, + "Execution profile should have stage-specific memory"); + + // Verify baseline memory is different + Assert.assertEquals( + baselineConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + 8192, + "Baseline should have original memory"); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java new file mode 100644 index 00000000000..a2fadeb80c7 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java @@ -0,0 +1,242 @@ +/* + * 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.temporal.yarn; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; + +import com.google.common.base.Optional; +import com.google.common.eventbus.EventBus; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; + + +/** + * Tests for {@link DynamicScalingYarnService} initialization and execution worker profile setup. + * Focuses on verifying that dynamic scaling correctly initializes execution worker profiles. + */ +public class DynamicScalingInitializationTest { + + private Config baseConfig; + private YarnConfiguration yarnConfiguration; + private FileSystem mockFileSystem; + private EventBus eventBus; + private AMRMClientAsync mockAMRMClient; + private RegisterApplicationMasterResponse mockRegisterResponse; + private MockedStatic amrmClientMockStatic; + + @BeforeMethod + public void setup() throws Exception { + baseConfig = ConfigFactory.empty() + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + ConfigValueFactory.fromAnyRef(8192)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, + ConfigValueFactory.fromAnyRef(4)) + .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, + ConfigValueFactory.fromAnyRef(1)) + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, + ConfigValueFactory.fromAnyRef(true)); + + yarnConfiguration = new YarnConfiguration(); + mockFileSystem = Mockito.mock(FileSystem.class); + eventBus = new EventBus("DynamicScalingInitializationTest"); + + mockAMRMClient = Mockito.mock(AMRMClientAsync.class); + mockRegisterResponse = Mockito.mock(RegisterApplicationMasterResponse.class); + + amrmClientMockStatic = Mockito.mockStatic(AMRMClientAsync.class); + amrmClientMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) + .thenReturn(mockAMRMClient); + + Mockito.doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); + Mockito.when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) + .thenReturn(mockRegisterResponse); + Mockito.when(mockRegisterResponse.getMaximumResourceCapability()) + .thenReturn(Resource.newInstance(102400, 32)); + } + + @AfterMethod + public void tearDown() { + if (amrmClientMockStatic != null) { + amrmClientMockStatic.close(); + } + } + + /** + * Tests that DynamicScalingYarnService initializes execution worker profile when enabled. + */ + @Test + public void testDynamicScalingInitializesExecutionProfile() throws Exception { + // Execute + DynamicScalingYarnService service = new DynamicScalingYarnService( + baseConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + DynamicScalingYarnService serviceSpy = Mockito.spy(service); + Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); + + serviceSpy.startUp(); + + // Verify that execution worker profile was requested + Mockito.verify(serviceSpy, Mockito.atLeastOnce()) + .requestContainersForWorkerProfile(any(WorkerProfile.class), anyInt()); + } + + /** + * Tests that execution worker profile has correct worker class configured. + */ + @Test + public void testExecutionProfileHasCorrectWorkerClass() { + // Setup - config with stage-specific memory + Config configWithMemory = baseConfig + .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, + ConfigValueFactory.fromAnyRef("32768")); + + // Simulate profile creation (as done in DynamicScalingYarnService) + Config profileConfig = configWithMemory + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)) + .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + ConfigValueFactory.fromAnyRef("execution")) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + ConfigValueFactory.fromAnyRef(32768)); + + WorkerProfile executionProfile = new WorkerProfile("initial-execution", profileConfig); + + // Verify + Assert.assertEquals(executionProfile.getName(), "initial-execution"); + Assert.assertEquals( + executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + "Execution profile should have ExecutionWorker class"); + Assert.assertEquals( + executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + 32768, + "Execution profile should have stage-specific memory"); + } + + /** + * Tests that dynamic scaling is NOT initialized when disabled. + */ + @Test + public void testDynamicScalingNotInitializedWhenDisabled() throws Exception { + // Setup - config with dynamic scaling disabled + Config disabledConfig = baseConfig + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, + ConfigValueFactory.fromAnyRef(false)); + + // Execute + YarnService service = new YarnService( + disabledConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + YarnService serviceSpy = Mockito.spy(service); + Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); + + serviceSpy.startUp(); + + // Verify - should request baseline containers, not execution profile + Mockito.verify(serviceSpy, Mockito.times(1)) + .requestContainers(anyInt(), any(Resource.class), any(Optional.class)); + } + + /** + * Tests that execution profile uses stage-specific memory when configured. + */ + @Test + public void testExecutionProfileUsesStageSpecificMemory() { + // Setup + int stageMemoryMb = 65536; + Config configWithStageMemory = baseConfig + .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, + ConfigValueFactory.fromAnyRef(String.valueOf(stageMemoryMb))); + + // Verify config has stage-specific memory + Assert.assertTrue(configWithStageMemory.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), + "Config should have stage-specific memory"); + Assert.assertEquals( + configWithStageMemory.getString(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), + String.valueOf(stageMemoryMb), + "Stage-specific memory should be set correctly"); + } + + /** + * Tests that execution profile falls back to baseline memory when not configured. + */ + @Test + public void testExecutionProfileFallsBackToBaselineMemory() { + // baseConfig doesn't have stage-specific memory + Assert.assertFalse(baseConfig.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), + "Config should not have stage-specific memory"); + + // Verify baseline memory is used + int baselineMemory = baseConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); + Assert.assertEquals(baselineMemory, 8192, + "Should use baseline memory when stage-specific not configured"); + } + + /** + * Tests that execution profile has correct Helix tag. + */ + @Test + public void testExecutionProfileHasCorrectHelixTag() { + // Simulate profile with Helix tag + Config profileConfig = baseConfig + .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + ConfigValueFactory.fromAnyRef("execution")); + + Assert.assertEquals( + profileConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), + "execution", + "Execution profile should have 'execution' Helix tag"); + } + + /** + * Tests that initial scaling directive has correct set point. + */ + @Test + public void testInitialScalingDirectiveSetPoint() throws Exception { + // Execute + DynamicScalingYarnService service = new DynamicScalingYarnService( + baseConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + DynamicScalingYarnService serviceSpy = Mockito.spy(service); + Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); + + serviceSpy.startUp(); + + // Verify - should request 1 execution container (set point = 1) + Mockito.verify(serviceSpy, Mockito.atLeastOnce()) + .requestContainersForWorkerProfile(any(WorkerProfile.class), Mockito.eq(1)); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java new file mode 100644 index 00000000000..fbd57c769fc --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java @@ -0,0 +1,232 @@ +/* + * 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.temporal.yarn; + +import java.util.Arrays; +import java.util.List; + +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.ProfileOverlay; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + + +/** + * Tests for ExecutionWorker profile overlay creation and configuration. + * Verifies that execution worker profiles are correctly configured with + * worker class, memory, and other settings. + */ +public class ExecutionWorkerProfileTest { + + private Config baseConfig; + + @BeforeMethod + public void setup() { + baseConfig = ConfigFactory.empty() + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + ConfigValueFactory.fromAnyRef(8192)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, + ConfigValueFactory.fromAnyRef(4)); + } + + /** + * Tests that execution profile overlay sets ExecutionWorker class. + */ + @Test + public void testExecutionProfileSetsWorkerClass() { + // Create overlay pairs as DynamicScalingYarnService does + ProfileOverlay.KVPair workerClassPair = new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(workerClassPair); + Config overlaidConfig = overlay.applyOverlay(baseConfig); + + // Verify + Assert.assertTrue(overlaidConfig.hasPath(GobblinTemporalConfigurationKeys.WORKER_CLASS), + "Overlaid config should have worker class"); + Assert.assertEquals(overlaidConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + "Should set ExecutionWorker class"); + } + + /** + * Tests that execution profile overlay sets Helix tag. + */ + @Test + public void testExecutionProfileSetsHelixTag() { + // Create overlay + ProfileOverlay.KVPair helixTagPair = new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "execution" + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(helixTagPair); + Config overlaidConfig = overlay.applyOverlay(baseConfig); + + // Verify + Assert.assertTrue(overlaidConfig.hasPath(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), + "Overlaid config should have Helix tag"); + Assert.assertEquals(overlaidConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), + "execution", + "Should set execution tag"); + } + + /** + * Tests that execution profile overlay can override memory configuration. + */ + @Test + public void testExecutionProfileOverridesMemory() { + // Setup - config with stage-specific memory + Config configWithStageMemory = baseConfig + .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, + ConfigValueFactory.fromAnyRef("65536")); + + // Create overlay with memory override + ProfileOverlay.KVPair memoryPair = new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + "65536" + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(memoryPair); + Config overlaidConfig = overlay.applyOverlay(baseConfig); + + // Verify + Assert.assertEquals(overlaidConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + "65536", + "Should override memory to 64GB for execution workers"); + } + + /** + * Tests complete execution profile overlay with all settings. + */ + @Test + public void testCompleteExecutionProfileOverlay() { + // Setup - simulate DynamicScalingYarnService.createExecutionProfileOverlay() + Config configWithStageMemory = baseConfig + .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, + ConfigValueFactory.fromAnyRef("32768")); + + // Create complete overlay + List overlayPairs = Arrays.asList( + new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ), + new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "execution" + ), + new ProfileOverlay.KVPair( + GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, + "32768" + ) + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(overlayPairs); + Config overlaidConfig = overlay.applyOverlay(baseConfig); + + // Verify all settings + Assert.assertEquals(overlaidConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + "Should set ExecutionWorker class"); + Assert.assertEquals(overlaidConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), + "execution", + "Should set execution tag"); + Assert.assertEquals(overlaidConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + "32768", + "Should set execution memory"); + Assert.assertEquals(overlaidConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), + 4, + "Should preserve baseline cores"); + } + + /** + * Tests that execution profile falls back to baseline memory when stage-specific not configured. + */ + @Test + public void testExecutionProfileFallsBackToBaselineMemory() { + // Create overlay without memory (simulating no stage-specific memory config) + List overlayPairs = Arrays.asList( + new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ), + new ProfileOverlay.KVPair( + GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, + "execution" + ) + ); + + ProfileOverlay overlay = new ProfileOverlay.Adding(overlayPairs); + Config overlaidConfig = overlay.applyOverlay(baseConfig); + + // Verify baseline memory is preserved + Assert.assertEquals(overlaidConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), + 8192, + "Should use baseline memory when stage-specific not configured"); + } + + /** + * Tests that baseline profile does NOT have ExecutionWorker class. + */ + @Test + public void testBaselineProfileDoesNotHaveExecutionWorker() { + // Baseline config should not have execution worker class + Config baselineWithWorker = baseConfig + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); + + Assert.assertNotEquals( + baselineWithWorker.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, + "Baseline profile should not use ExecutionWorker"); + } + + /** + * Tests that execution profile overlay is idempotent. + */ + @Test + public void testExecutionProfileOverlayIsIdempotent() { + // Create overlay + ProfileOverlay.KVPair workerClassPair = new ProfileOverlay.KVPair( + GobblinTemporalConfigurationKeys.WORKER_CLASS, + GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS + ); + ProfileOverlay overlay = new ProfileOverlay.Adding(workerClassPair); + + // Apply twice + Config overlaidOnce = overlay.applyOverlay(baseConfig); + Config overlaidTwice = overlay.applyOverlay(overlaidOnce); + + // Verify same result + Assert.assertEquals( + overlaidOnce.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + overlaidTwice.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), + "Overlay should be idempotent"); + } +} From bb49427e265ffdcd78efd85da784aa60e1064794 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Mon, 15 Dec 2025 22:59:53 +0530 Subject: [PATCH 09/13] - only forward NestingExecWorkflow to execution queue --- .../impl/ExecuteGobblinWorkflowImpl.java | 16 +- .../impl/ProcessWorkUnitsWorkflowImpl.java | 26 +++- .../ProcessWorkUnitsWorkflowImplTest.java | 145 ++++++++++++++++++ 3 files changed, 167 insertions(+), 20 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index d6ba8dd8847..87deb1baf7a 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -161,20 +161,12 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event } protected ProcessWorkUnitsWorkflow createProcessWorkUnitsWorkflow(Properties jobProps) { - Config config = ConfigFactory.parseProperties(jobProps); - boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) - && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); - - ChildWorkflowOptions.Builder childOpts = ChildWorkflowOptions.newBuilder() + ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_TERMINATE) .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(PROCESS_WORKFLOW_ID_BASE, ConfigFactory.parseProperties(jobProps))) - .setSearchAttributes(TemporalWorkFlowUtils.generateGaasSearchAttributes(jobProps)); - - if (dynamicScalingEnabled) { - childOpts.setTaskQueue(WorkflowStage.WORK_EXECUTION.getTaskQueue(config)); - } - - return Workflow.newChildWorkflowStub(ProcessWorkUnitsWorkflow.class, childOpts.build()); + .setSearchAttributes(TemporalWorkFlowUtils.generateGaasSearchAttributes(jobProps)) + .build(); + return Workflow.newChildWorkflowStub(ProcessWorkUnitsWorkflow.class, childOpts); } protected TimeBudget calcWUProcTimeBudget(Instant jobStartTime, WorkUnitsSizeSummary wuSizeSummary, Properties jobProps) { diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java index 855ea437776..f5fb0f044dc 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImpl.java @@ -134,25 +134,35 @@ protected Workload createWorkload(WUProcessingSpec workSpec, protected NestingExecWorkflow createProcessingWorkflow(FileSystemJobStateful f, Map searchAttributes) { - ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() + Config config = WorkerConfig.of(this).orElse(ConfigFactory.empty()); + boolean dynamicScalingEnabled = config.hasPath(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED) + && config.getBoolean(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED); + + ChildWorkflowOptions.Builder childOpts = ChildWorkflowOptions.newBuilder() .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_TERMINATE) .setSearchAttributes(searchAttributes) - .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(CHILD_WORKFLOW_ID_BASE, f, - WorkerConfig.of(this).orElse(ConfigFactory.empty()))) - .build(); - // TODO: to incorporate multiple different concrete `NestingExecWorkflow` sub-workflows in the same super-workflow... shall we use queues?!?!? - return Workflow.newChildWorkflowStub(NestingExecWorkflow.class, childOpts); + .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(CHILD_WORKFLOW_ID_BASE, f, config)); + + // Route NestingExecWorkflow (work execution) to execution + if (dynamicScalingEnabled) { + childOpts.setTaskQueue(config.hasPath(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) + ? config.getString(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) + : GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE); + } + + return Workflow.newChildWorkflowStub(NestingExecWorkflow.class, childOpts.build()); } protected CommitStepWorkflow createCommitStepWorkflow(Map searchAttributes) { + Config config = WorkerConfig.of(this).orElse(ConfigFactory.empty()); ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder() // TODO: verify to instead use: Policy.PARENT_CLOSE_POLICY_TERMINATE) .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_ABANDON) .setSearchAttributes(searchAttributes) - .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, - WorkerConfig.of(this).orElse(ConfigFactory.empty()))) + .setWorkflowId(Help.qualifyNamePerExecWithFlowExecId(COMMIT_STEP_WORKFLOW_ID_BASE, config)) .build(); + // CommitStepWorkflow inherits default queue from ProcessWorkUnitsWorkflow parent return Workflow.newChildWorkflowStub(CommitStepWorkflow.class, childOpts); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java index 6af045d70ef..946149e34fe 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java @@ -34,8 +34,11 @@ import io.temporal.workflow.ChildWorkflowOptions; import io.temporal.workflow.Workflow; +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.cluster.WorkerConfig; +import org.apache.gobblin.temporal.ddm.work.WUProcessingSpec; import org.apache.gobblin.temporal.ddm.workflow.CommitStepWorkflow; +import org.apache.gobblin.temporal.util.nesting.workflow.NestingExecWorkflow; /** @@ -163,4 +166,146 @@ public void testCommitStepWorkflowIdQualification() { workflowMockedStatic.verify(() -> Workflow.newChildWorkflowStub( Mockito.any(), Mockito.any()), Mockito.times(1)); } + + /** + * Tests that NestingExecWorkflow is routed to execution queue when dynamic scaling is enabled. + */ + @Test + public void testCreateProcessingWorkflowWithDynamicScalingEnabled() { + // Setup + Map searchAttributes = new HashMap<>(); + WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); + + Config mockConfig = ConfigFactory.parseString( + GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=true\n" + + GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE + "=TestExecutionQueue" + ); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); + + // Capture the ChildWorkflowOptions passed to newChildWorkflowStub + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(NestingExecWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + // Verify task queue IS set to execution queue + Assert.assertEquals(options.getTaskQueue(), "TestExecutionQueue", + "NestingExecWorkflow should use execution queue when dynamic scaling is enabled"); + return mockNestingWorkflow; + }); + + // Execute + NestingExecWorkflow result = workflow.createProcessingWorkflow(mockSpec, searchAttributes); + + // Verify + Assert.assertNotNull(result); + workflowMockedStatic.verify(() -> Workflow.newChildWorkflowStub( + Mockito.eq(NestingExecWorkflow.class), + Mockito.any(ChildWorkflowOptions.class)), Mockito.times(1)); + } + + /** + * Tests that NestingExecWorkflow uses default execution queue when config key is not present. + */ + @Test + public void testCreateProcessingWorkflowWithDynamicScalingEnabledDefaultQueue() { + // Setup + Map searchAttributes = new HashMap<>(); + WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); + + Config mockConfig = ConfigFactory.parseString( + GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=true" + ); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); + + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(NestingExecWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + // Verify task queue uses default execution queue + Assert.assertEquals(options.getTaskQueue(), + GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, + "NestingExecWorkflow should use default execution queue"); + return mockNestingWorkflow; + }); + + // Execute + workflow.createProcessingWorkflow(mockSpec, searchAttributes); + } + + /** + * Tests that NestingExecWorkflow does NOT have task queue set when dynamic scaling is disabled. + */ + @Test + public void testCreateProcessingWorkflowWithDynamicScalingDisabled() { + // Setup + Map searchAttributes = new HashMap<>(); + WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); + + Config mockConfig = ConfigFactory.parseString( + GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=false" + ); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); + + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(NestingExecWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + // Verify task queue is NOT set (inherits from parent) + Assert.assertNull(options.getTaskQueue(), + "NestingExecWorkflow should not have explicit task queue when dynamic scaling is disabled"); + return mockNestingWorkflow; + }); + + // Execute + NestingExecWorkflow result = workflow.createProcessingWorkflow(mockSpec, searchAttributes); + + // Verify + Assert.assertNotNull(result); + } + + /** + * Tests that NestingExecWorkflow does NOT have task queue set when dynamic scaling config is absent. + */ + @Test + public void testCreateProcessingWorkflowWithoutDynamicScalingConfig() { + // Setup + Map searchAttributes = new HashMap<>(); + WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); + + Config mockConfig = ConfigFactory.empty(); + + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) + .thenReturn(java.util.Optional.of(mockConfig)); + + NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); + + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( + Mockito.eq(NestingExecWorkflow.class), + Mockito.any(ChildWorkflowOptions.class))) + .thenAnswer(invocation -> { + ChildWorkflowOptions options = invocation.getArgument(1); + // Verify task queue is NOT set (inherits from parent) + Assert.assertNull(options.getTaskQueue(), + "NestingExecWorkflow should not have explicit task queue when dynamic scaling config is absent"); + return mockNestingWorkflow; + }); + + // Execute + workflow.createProcessingWorkflow(mockSpec, searchAttributes); + } } From 85959e3c16b6a4cd9b5afe85c27faf9a02223d3d Mon Sep 17 00:00:00 2001 From: agam-99 Date: Tue, 16 Dec 2025 11:38:46 +0530 Subject: [PATCH 10/13] removed redundant unit tests --- .../RecommendScalingForWorkUnits.java | 1 + .../impl/ExecuteGobblinWorkflowImpl.java | 2 +- .../yarn/DynamicScalingYarnService.java | 2 - .../GobblinTemporalTaskRunnerTest.java | 153 ++++----- .../ddm/worker/ExecutionWorkerTest.java | 243 ++++++++++++--- .../ddm/workflow/WorkflowStageTest.java | 75 ----- .../ProcessWorkUnitsWorkflowImplTest.java | 110 ++----- .../TaskQueueRoutingIntegrationTest.java | 289 ----------------- ...DummyDynamicScalingYarnServiceManager.java | 38 --- .../DynamicScalingInitializationTest.java | 242 --------------- .../DynamicScalingYarnServiceManagerTest.java | 133 -------- .../yarn/DynamicScalingYarnServiceTest.java | 291 ------------------ .../yarn/ExecutionWorkerProfileTest.java | 232 -------------- .../temporal/yarn/YarnServiceTest.java | 129 -------- 14 files changed, 304 insertions(+), 1636 deletions(-) delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java delete mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java index 059abd72444..f3715a0d98e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/RecommendScalingForWorkUnits.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.gobblin.temporal.ddm.activity; import java.util.List; diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java index 87deb1baf7a..05fa2a8546b 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/workflow/impl/ExecuteGobblinWorkflowImpl.java @@ -52,7 +52,6 @@ import org.apache.gobblin.temporal.ddm.activity.GenerateWorkUnits; import org.apache.gobblin.temporal.ddm.activity.RecommendScalingForWorkUnits; import org.apache.gobblin.temporal.ddm.launcher.ProcessWorkUnitsJobLauncher; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; import org.apache.gobblin.temporal.ddm.util.JobStateUtils; import org.apache.gobblin.temporal.ddm.util.TemporalWorkFlowUtils; import org.apache.gobblin.temporal.ddm.work.CommitStats; @@ -112,6 +111,7 @@ public ExecGobblinStats execute(Properties jobProps, EventSubmitterContext event ActivityType.RECOMMEND_SCALING.buildActivityOptions(temporalJobProps, false)); List scalingDirectives = recommendScalingStub.recommendScaling(wuSizeSummary, generateWorkUnitResult.getSourceClass(), timeBudget, jobProps); + log.info("Recommended scaling to process WUs within {}: {}", timeBudget, scalingDirectives); try { ScalingDirectivesRecipient recipient = createScalingDirectivesRecipient(jobProps, closer); List adjustedScalingDirectives = adjustRecommendedScaling(jobProps, scalingDirectives); diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java index 0d7645c76cf..0010a45ff8e 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnService.java @@ -27,7 +27,6 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.gobblin.util.ConfigUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -75,7 +74,6 @@ public DynamicScalingYarnService(Config config, String applicationName, String a this.actualWorkforceStaffing = WorkforceStaffing.initialize(0); this.workforcePlan = new WorkforcePlan(this.config, this.config.getInt(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY)); - this.removedContainerIds = new ConcurrentLinkedQueue<>(); this.profileNameSuffixGenerator = new AtomicLong(); } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java index 92582fedfa2..69772116d2a 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/cluster/GobblinTemporalTaskRunnerTest.java @@ -17,6 +17,12 @@ package org.apache.gobblin.temporal.cluster; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -25,116 +31,117 @@ import com.typesafe.config.ConfigValueFactory; import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.ddm.worker.ExecutionWorker; -import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldWorker; +import org.apache.gobblin.temporal.workflows.service.ManagedWorkflowServiceStubs; /** - * Tests for {@link GobblinTemporalTaskRunner} focusing on worker class resolution - * from configuration without relying on system properties. + * Tests for {@link GobblinTemporalTaskRunner} worker initialization logic. */ public class GobblinTemporalTaskRunnerTest { /** - * Tests that worker class is correctly read from config (not system properties). - * This verifies the fix where we removed System.getProperty() in favor of ConfigUtils.getString(). + * Tests that initializeExecutionWorkers does nothing when dynamic scaling is disabled. */ @Test - public void testWorkerClassResolvedFromConfig() { - // Setup - config with ExecutionWorker class + public void testInitializeExecutionWorkersWhenDynamicScalingDisabled() throws Exception { Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, ConfigValueFactory.fromAnyRef(false)) .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(ExecutionWorker.class.getName())); + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); + + GobblinTemporalTaskRunner taskRunner = createMockTaskRunner(config); + List workers = getWorkersField(taskRunner); + int initialWorkerCount = workers.size(); + + invokeInitializeExecutionWorkers(taskRunner); - // Verify the config contains the expected worker class - String workerClass = config.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS); - Assert.assertEquals(workerClass, ExecutionWorker.class.getName(), - "Config should contain ExecutionWorker class name"); + Assert.assertEquals(workers.size(), initialWorkerCount, + "No workers should be added when dynamic scaling is disabled"); } /** - * Tests that default worker class is used when not configured. + * Tests that initializeExecutionWorkers does nothing when container is already ExecutionWorker. */ @Test - public void testDefaultWorkerClassWhenNotConfigured() { - // Setup - empty config - Config config = ConfigFactory.empty(); - - // Verify default is used - String workerClass = config.hasPath(GobblinTemporalConfigurationKeys.WORKER_CLASS) - ? config.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS) - : GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS; - - Assert.assertEquals(workerClass, HelloWorldWorker.class.getName(), - "Should use default HelloWorldWorker when not configured"); + public void testInitializeExecutionWorkersWhenAlreadyExecutionWorker() throws Exception { + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, ConfigValueFactory.fromAnyRef(true)) + .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)); + + GobblinTemporalTaskRunner taskRunner = createMockTaskRunner(config); + List workers = getWorkersField(taskRunner); + int initialWorkerCount = workers.size(); + + invokeInitializeExecutionWorkers(taskRunner); + + Assert.assertEquals(workers.size(), initialWorkerCount, + "No workers should be added when container is already ExecutionWorker"); } /** - * Tests that worker class configuration is properly overridden in profile overlay. - * This simulates how ExecutionWorker profile overrides the baseline worker class. + * Tests that initializeExecutionWorkers does nothing when dynamic scaling config is missing. */ @Test - public void testWorkerClassOverrideInProfile() { - // Setup - baseline config with default worker - Config baselineConfig = ConfigFactory.empty() + public void testInitializeExecutionWorkersWhenConfigMissing() throws Exception { + Config config = ConfigFactory.empty() .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(HelloWorldWorker.class.getName())); + ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); - // Simulate profile overlay for execution worker - Config executionConfig = baselineConfig - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(ExecutionWorker.class.getName())); + GobblinTemporalTaskRunner taskRunner = createMockTaskRunner(config); + List workers = getWorkersField(taskRunner); + int initialWorkerCount = workers.size(); - // Verify baseline uses HelloWorldWorker - Assert.assertEquals(baselineConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - HelloWorldWorker.class.getName(), - "Baseline should use HelloWorldWorker"); + invokeInitializeExecutionWorkers(taskRunner); - // Verify execution profile uses ExecutionWorker - Assert.assertEquals(executionConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - ExecutionWorker.class.getName(), - "Execution profile should use ExecutionWorker"); + Assert.assertEquals(workers.size(), initialWorkerCount, + "No workers should be added when dynamic scaling config is missing"); } /** - * Tests that EXECUTION_WORKER_CLASS constant matches ExecutionWorker class name. + * Helper to create a mock GobblinTemporalTaskRunner with necessary fields set. */ - @Test - public void testExecutionWorkerClassConstant() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - ExecutionWorker.class.getName(), - "EXECUTION_WORKER_CLASS constant should match ExecutionWorker.class.getName()"); + private GobblinTemporalTaskRunner createMockTaskRunner(Config config) throws Exception { + GobblinTemporalTaskRunner taskRunner = Mockito.mock(GobblinTemporalTaskRunner.class, Mockito.CALLS_REAL_METHODS); + + // Set clusterConfig field + Field clusterConfigField = GobblinTemporalTaskRunner.class.getDeclaredField("clusterConfig"); + clusterConfigField.setAccessible(true); + clusterConfigField.set(taskRunner, config); + + // Set workers list + Field workersField = GobblinTemporalTaskRunner.class.getDeclaredField("workers"); + workersField.setAccessible(true); + workersField.set(taskRunner, new ArrayList()); + + // Mock managedWorkflowServiceStubs + ManagedWorkflowServiceStubs mockStubs = Mockito.mock(ManagedWorkflowServiceStubs.class); + Field stubsField = GobblinTemporalTaskRunner.class.getDeclaredField("managedWorkflowServiceStubs"); + stubsField.setAccessible(true); + stubsField.set(taskRunner, mockStubs); + + // Mock WorkflowClient + Mockito.when(mockStubs.getWorkflowServiceStubs()).thenReturn(null); + + return taskRunner; } /** - * Tests that DEFAULT_WORKER_CLASS constant matches HelloWorldWorker class name. + * Helper to invoke the private initializeExecutionWorkers method using reflection. */ - @Test - public void testDefaultWorkerClassConstant() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS, - HelloWorldWorker.class.getName(), - "DEFAULT_WORKER_CLASS constant should match HelloWorldWorker.class.getName()"); + private void invokeInitializeExecutionWorkers(GobblinTemporalTaskRunner taskRunner) throws Exception { + Method method = GobblinTemporalTaskRunner.class.getDeclaredMethod("initializeExecutionWorkers"); + method.setAccessible(true); + method.invoke(taskRunner); } /** - * Tests worker class configuration for different container types in dynamic scaling. + * Helper to get the workers list field using reflection. */ - @Test - public void testWorkerClassForDifferentContainerTypes() { - // Baseline container config (WorkFulfillmentWorker) - Config baselineConfig = ConfigFactory.empty() - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); - - // Execution container config (ExecutionWorker) - Config executionConfig = ConfigFactory.empty() - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)); - - // Verify different worker classes for different container types - Assert.assertNotEquals( - baselineConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - executionConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - "Baseline and execution containers should use different worker classes"); + @SuppressWarnings("unchecked") + private List getWorkersField(GobblinTemporalTaskRunner taskRunner) throws Exception { + Field workersField = GobblinTemporalTaskRunner.class.getDeclaredField("workers"); + workersField.setAccessible(true); + return (List) workersField.get(taskRunner); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java index 76110aad035..7d235b3c9df 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java @@ -17,98 +17,245 @@ package org.apache.gobblin.temporal.ddm.worker; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import org.mockito.Mockito; import org.testng.Assert; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigValueFactory; +import io.temporal.client.WorkflowClient; +import io.temporal.worker.WorkerOptions; + import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.ddm.activity.impl.ProcessWorkUnitImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.NestingExecOfProcessWorkUnitWorkflowImpl; +import org.apache.gobblin.temporal.ddm.workflow.impl.ProcessWorkUnitsWorkflowImpl; +import org.apache.gobblin.util.ConfigUtils; /** - * Tests for {@link ExecutionWorker} configuration verification. - * Tests configuration keys and default values without requiring Temporal infrastructure. + * Tests for {@link ExecutionWorker} verifying workflow/activity registration and configuration. */ public class ExecutionWorkerTest { - private Config baseConfig; + /** + * Tests that ExecutionWorker registers only the workflows needed for work execution. + */ + @Test + public void testGetWorkflowImplClasses() throws Exception { + Config config = ConfigFactory.empty(); + ExecutionWorker worker = createMockWorker(config); + + Class[] workflows = invokeGetWorkflowImplClasses(worker); + + Assert.assertEquals(workflows.length, 2, + "ExecutionWorker should register exactly 2 workflow types"); - @BeforeMethod - public void setup() { - baseConfig = ConfigFactory.empty() - .withValue(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, - ConfigValueFactory.fromAnyRef("TestQueue")); + List workflowNames = Arrays.stream(workflows) + .map(Class::getName) + .collect(Collectors.toList()); + + Assert.assertTrue(workflowNames.contains(ProcessWorkUnitsWorkflowImpl.class.getName()), + "ExecutionWorker should register ProcessWorkUnitsWorkflowImpl"); + Assert.assertTrue(workflowNames.contains(NestingExecOfProcessWorkUnitWorkflowImpl.class.getName()), + "ExecutionWorker should register NestingExecOfProcessWorkUnitWorkflowImpl"); } + /** + * Tests that ExecutionWorker registers only ProcessWorkUnit activity. + */ @Test - public void testExecutionWorkerUsesExecutionTaskQueue() { - String executionQueue = "GobblinTemporalExecutionQueue"; - Config config = baseConfig + public void testGetActivityImplInstances() throws Exception { + Config config = ConfigFactory.empty(); + ExecutionWorker worker = createMockWorker(config); + + Object[] activities = invokeGetActivityImplInstances(worker); + + Assert.assertEquals(activities.length, 1, + "ExecutionWorker should register exactly 1 activity type"); + Assert.assertTrue(activities[0] instanceof ProcessWorkUnitImpl, + "ExecutionWorker should register ProcessWorkUnitImpl activity"); + } + + /** + * Tests that ExecutionWorker uses execution task queue from config. + */ + @Test + public void testGetTaskQueueFromConfig() throws Exception { + String customQueue = "CustomExecutionQueue"; + Config config = ConfigFactory.empty() .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, - ConfigValueFactory.fromAnyRef(executionQueue)); + ConfigValueFactory.fromAnyRef(customQueue)); + + ExecutionWorker worker = createMockWorker(config); + String taskQueue = invokeGetTaskQueue(worker); - String configuredQueue = config.getString(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE); - Assert.assertEquals(configuredQueue, executionQueue); + Assert.assertEquals(taskQueue, customQueue, + "ExecutionWorker should use execution task queue from config"); } + /** + * Tests that ExecutionWorker uses default execution task queue when not configured. + */ @Test - public void testExecutionWorkerUsesDefaultExecutionQueue() { - String defaultQueue = baseConfig.hasPath(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) - ? baseConfig.getString(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE) - : GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE; + public void testGetTaskQueueDefault() throws Exception { + Config config = ConfigFactory.empty(); + + ExecutionWorker worker = createMockWorker(config); + String taskQueue = invokeGetTaskQueue(worker); - Assert.assertEquals(defaultQueue, GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE); + Assert.assertEquals(taskQueue, GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, + "ExecutionWorker should use default execution task queue when not configured"); } + /** + * Tests that ExecutionWorker creates WorkerOptions with correct concurrency settings. + */ @Test - public void testExecutionWorkerRegistersCorrectWorkflows() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - "org.apache.gobblin.temporal.ddm.worker.ExecutionWorker"); + public void testCreateWorkerOptionsWithCustomConcurrency() throws Exception { + int customConcurrency = 10; + Config config = ConfigFactory.empty() + .withValue(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + ConfigValueFactory.fromAnyRef(customConcurrency)); + + ExecutionWorker worker = createMockWorker(config); + WorkerOptions options = invokeCreateWorkerOptions(worker); + + Assert.assertEquals(options.getMaxConcurrentActivityExecutionSize(), customConcurrency, + "MaxConcurrentActivityExecutionSize should match configured value"); + Assert.assertEquals(options.getMaxConcurrentLocalActivityExecutionSize(), customConcurrency, + "MaxConcurrentLocalActivityExecutionSize should match configured value"); + Assert.assertEquals(options.getMaxConcurrentWorkflowTaskExecutionSize(), customConcurrency, + "MaxConcurrentWorkflowTaskExecutionSize should match configured value"); } + /** + * Tests that ExecutionWorker creates WorkerOptions with default concurrency. + */ @Test - public void testExecutionWorkerRegistersOnlyProcessWorkUnitActivity() { - Assert.assertTrue(baseConfig.hasPath(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE)); + public void testCreateWorkerOptionsWithDefaultConcurrency() throws Exception { + Config config = ConfigFactory.empty(); + + ExecutionWorker worker = createMockWorker(config); + WorkerOptions options = invokeCreateWorkerOptions(worker); + + int defaultConcurrency = GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER; + Assert.assertEquals(options.getMaxConcurrentActivityExecutionSize(), defaultConcurrency, + "MaxConcurrentActivityExecutionSize should use default value"); + Assert.assertEquals(options.getMaxConcurrentLocalActivityExecutionSize(), defaultConcurrency, + "MaxConcurrentLocalActivityExecutionSize should use default value"); + Assert.assertEquals(options.getMaxConcurrentWorkflowTaskExecutionSize(), defaultConcurrency, + "MaxConcurrentWorkflowTaskExecutionSize should use default value"); } + /** + * Tests that ExecutionWorker sets deadlock detection timeout correctly. + */ @Test - public void testExecutionWorkerConfiguresWorkerOptions() { - int expectedConcurrency = 10; - Config config = baseConfig + public void testCreateWorkerOptionsDeadlockTimeout() throws Exception { + Config config = ConfigFactory.empty(); + + ExecutionWorker worker = createMockWorker(config); + WorkerOptions options = invokeCreateWorkerOptions(worker); + + long expectedTimeoutMillis = TimeUnit.SECONDS.toMillis(ExecutionWorker.DEADLOCK_DETECTION_TIMEOUT_SECONDS); + Assert.assertEquals(options.getDefaultDeadlockDetectionTimeout(), expectedTimeoutMillis, + "Deadlock detection timeout should be set correctly"); + } + + /** + * Tests that maxExecutionConcurrency field is initialized from config. + */ + @Test + public void testMaxExecutionConcurrencyInitialization() throws Exception { + int customConcurrency = 15; + Config config = ConfigFactory.empty() .withValue(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, - ConfigValueFactory.fromAnyRef(expectedConcurrency)); + ConfigValueFactory.fromAnyRef(customConcurrency)); - int configuredConcurrency = config.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER); - Assert.assertEquals(configuredConcurrency, expectedConcurrency); + ExecutionWorker worker = createMockWorker(config); + + Assert.assertEquals(worker.maxExecutionConcurrency, customConcurrency, + "maxExecutionConcurrency should be initialized from config"); } + /** + * Tests that maxExecutionConcurrency uses default when not configured. + */ @Test - public void testExecutionWorkerUsesDefaultConcurrency() { - int defaultConcurrency = GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER; + public void testMaxExecutionConcurrencyDefault() throws Exception { + Config config = ConfigFactory.empty(); + + ExecutionWorker worker = createMockWorker(config); + + Assert.assertEquals(worker.maxExecutionConcurrency, + GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER, + "maxExecutionConcurrency should use default value when not configured"); + } + + /** + * Helper to create a mock ExecutionWorker without calling the constructor. + */ + private ExecutionWorker createMockWorker(Config config) throws Exception { + ExecutionWorker worker = Mockito.mock(ExecutionWorker.class, Mockito.CALLS_REAL_METHODS); + + // Set config field + Field configField = org.apache.gobblin.temporal.cluster.AbstractTemporalWorker.class.getDeclaredField("config"); + configField.setAccessible(true); + configField.set(worker, config); - int concurrency = baseConfig.hasPath(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER) - ? baseConfig.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER) - : defaultConcurrency; + // Set maxExecutionConcurrency field + Field maxConcurrencyField = ExecutionWorker.class.getDeclaredField("maxExecutionConcurrency"); + maxConcurrencyField.setAccessible(true); + int concurrency = ConfigUtils.getInt(config, GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, + GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER); + maxConcurrencyField.set(worker, concurrency); - Assert.assertEquals(concurrency, defaultConcurrency); + return worker; } - @Test - public void testExecutionWorkerSetsDeadlockDetectionTimeout() { - Assert.assertTrue(true); + /** + * Helper to invoke the protected getWorkflowImplClasses method using reflection. + */ + private Class[] invokeGetWorkflowImplClasses(ExecutionWorker worker) throws Exception { + Method method = ExecutionWorker.class.getDeclaredMethod("getWorkflowImplClasses"); + method.setAccessible(true); + return (Class[]) method.invoke(worker); } - @Test - public void testMaxExecutionConcurrencyInitialization() { - int expectedConcurrency = 15; - Config config = baseConfig - .withValue(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER, - ConfigValueFactory.fromAnyRef(expectedConcurrency)); + /** + * Helper to invoke the protected getActivityImplInstances method using reflection. + */ + private Object[] invokeGetActivityImplInstances(ExecutionWorker worker) throws Exception { + Method method = ExecutionWorker.class.getDeclaredMethod("getActivityImplInstances"); + method.setAccessible(true); + return (Object[]) method.invoke(worker); + } + + /** + * Helper to invoke the protected getTaskQueue method using reflection. + */ + private String invokeGetTaskQueue(ExecutionWorker worker) throws Exception { + Method method = ExecutionWorker.class.getDeclaredMethod("getTaskQueue"); + method.setAccessible(true); + return (String) method.invoke(worker); + } - int configuredConcurrency = config.getInt(GobblinTemporalConfigurationKeys.TEMPORAL_NUM_THREADS_PER_WORKER); - Assert.assertEquals(configuredConcurrency, expectedConcurrency); + /** + * Helper to invoke the protected createWorkerOptions method using reflection. + */ + private WorkerOptions invokeCreateWorkerOptions(ExecutionWorker worker) throws Exception { + Method method = ExecutionWorker.class.getDeclaredMethod("createWorkerOptions"); + method.setAccessible(true); + return (WorkerOptions) method.invoke(worker); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java index 2c6c91f1740..7c386caf169 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java @@ -124,79 +124,4 @@ public void testDifferentStagesUseDifferentQueues() { Assert.assertNotEquals(executionQueue, discoveryQueue, "Execution and discovery should use different queues"); } - - /** - * Tests that execution queue configuration is independent of default queue. - */ - @Test - public void testExecutionQueueIndependentOfDefaultQueue() { - // Setup - only configure execution queue - Config config = ConfigFactory.empty() - .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, - ConfigValueFactory.fromAnyRef("CustomExecutionQueue")); - - // Execute - String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); - String discoveryQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); - - // Verify - Assert.assertEquals(executionQueue, "CustomExecutionQueue", - "Should use custom execution queue"); - Assert.assertEquals(discoveryQueue, GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, - "Should use default queue for discovery"); - } - - /** - * Tests that default execution queue constant is correct. - */ - @Test - public void testDefaultExecutionQueueConstant() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, - "GobblinTemporalExecutionQueue", - "Default execution queue should be GobblinTemporalExecutionQueue"); - } - - /** - * Tests that default task queue constant is correct. - */ - @Test - public void testDefaultTaskQueueConstant() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, - "GobblinTemporalTaskQueue", - "Default task queue should be GobblinTemporalTaskQueue"); - } - - /** - * Tests task queue configuration for all workflow stages. - */ - @Test - public void testAllWorkflowStagesHaveTaskQueues() { - Config config = ConfigFactory.empty(); - - for (WorkflowStage stage : WorkflowStage.values()) { - String taskQueue = stage.getTaskQueue(config); - Assert.assertNotNull(taskQueue, "Stage " + stage + " should have a task queue"); - Assert.assertFalse(taskQueue.isEmpty(), "Stage " + stage + " task queue should not be empty"); - } - } - - /** - * Tests that execution queue config key is correct. - */ - @Test - public void testExecutionQueueConfigKey() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, - "gobblin.temporal.execution.task.queue.name", - "Execution task queue config key should match expected value"); - } - - /** - * Tests that task queue config key is correct. - */ - @Test - public void testTaskQueueConfigKey() { - Assert.assertEquals(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, - "gobblin.temporal.task.queue.name", - "Task queue config key should match expected value"); - } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java index 946149e34fe..272aca43fdf 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/impl/ProcessWorkUnitsWorkflowImplTest.java @@ -19,7 +19,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.Properties; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -37,6 +36,7 @@ import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; import org.apache.gobblin.temporal.cluster.WorkerConfig; import org.apache.gobblin.temporal.ddm.work.WUProcessingSpec; +import org.apache.gobblin.temporal.ddm.work.assistance.Help; import org.apache.gobblin.temporal.ddm.workflow.CommitStepWorkflow; import org.apache.gobblin.temporal.util.nesting.workflow.NestingExecWorkflow; @@ -49,12 +49,21 @@ public class ProcessWorkUnitsWorkflowImplTest { private MockedStatic workflowMockedStatic; private MockedStatic workerConfigMockedStatic; + private MockedStatic helpMockedStatic; private ProcessWorkUnitsWorkflowImpl workflow; @BeforeMethod public void setup() { workflowMockedStatic = Mockito.mockStatic(Workflow.class); workerConfigMockedStatic = Mockito.mockStatic(WorkerConfig.class); + helpMockedStatic = Mockito.mockStatic(Help.class); + + // Mock Help.qualifyNamePerExecWithFlowExecId to return a simple workflow ID + helpMockedStatic.when(() -> Help.qualifyNamePerExecWithFlowExecId(Mockito.anyString(), Mockito.any(), Mockito.any())) + .thenReturn("test-workflow-id"); + helpMockedStatic.when(() -> Help.qualifyNamePerExecWithFlowExecId(Mockito.anyString(), Mockito.any())) + .thenReturn("test-workflow-id"); + workflow = new ProcessWorkUnitsWorkflowImpl(); } @@ -66,6 +75,9 @@ public void tearDown() { if (workerConfigMockedStatic != null) { workerConfigMockedStatic.close(); } + if (helpMockedStatic != null) { + helpMockedStatic.close(); + } } /** @@ -78,13 +90,13 @@ public void testCreateCommitStepWorkflowUsesDefaultQueue() { // Setup Map searchAttributes = new HashMap<>(); searchAttributes.put("test", "value"); - + Config mockConfig = ConfigFactory.empty(); workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) .thenReturn(java.util.Optional.of(mockConfig)); CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); - + // Capture the ChildWorkflowOptions passed to newChildWorkflowStub workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( Mockito.eq(CommitStepWorkflow.class), @@ -92,7 +104,7 @@ public void testCreateCommitStepWorkflowUsesDefaultQueue() { .thenAnswer(invocation -> { ChildWorkflowOptions options = invocation.getArgument(1); // Verify task queue is NOT set (should be null to inherit from parent) - Assert.assertNull(options.getTaskQueue(), + Assert.assertNull(options.getTaskQueue(), "CommitStepWorkflow should not have explicit task queue set"); Assert.assertEquals(options.getSearchAttributes(), searchAttributes); return mockCommitWorkflow; @@ -115,12 +127,12 @@ public void testCreateCommitStepWorkflowUsesDefaultQueue() { public void testCreateCommitStepWorkflowWithoutWorkerConfig() { // Setup Map searchAttributes = new HashMap<>(); - + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) .thenReturn(java.util.Optional.empty()); CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); - + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( Mockito.eq(CommitStepWorkflow.class), Mockito.any(ChildWorkflowOptions.class))) @@ -133,40 +145,6 @@ public void testCreateCommitStepWorkflowWithoutWorkerConfig() { Assert.assertNotNull(result); } - /** - * Tests that workflow ID is properly qualified with flow execution ID. - */ - @Test - public void testCommitStepWorkflowIdQualification() { - // Setup - Map searchAttributes = new HashMap<>(); - Config mockConfig = ConfigFactory.parseString("flow.executionId=test-exec-123"); - - workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) - .thenReturn(java.util.Optional.of(mockConfig)); - - CommitStepWorkflow mockCommitWorkflow = Mockito.mock(CommitStepWorkflow.class); - - workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( - Mockito.eq(CommitStepWorkflow.class), - Mockito.any(ChildWorkflowOptions.class))) - .thenAnswer(invocation -> { - ChildWorkflowOptions options = invocation.getArgument(1); - String workflowId = options.getWorkflowId(); - Assert.assertNotNull(workflowId); - Assert.assertTrue(workflowId.contains("CommitStepWorkflow"), - "Workflow ID should contain base name"); - return mockCommitWorkflow; - }); - - // Execute - workflow.createCommitStepWorkflow(searchAttributes); - - // Verify through mock interactions - workflowMockedStatic.verify(() -> Workflow.newChildWorkflowStub( - Mockito.any(), Mockito.any()), Mockito.times(1)); - } - /** * Tests that NestingExecWorkflow is routed to execution queue when dynamic scaling is enabled. */ @@ -175,17 +153,17 @@ public void testCreateProcessingWorkflowWithDynamicScalingEnabled() { // Setup Map searchAttributes = new HashMap<>(); WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); - + Config mockConfig = ConfigFactory.parseString( GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=true\n" + GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE + "=TestExecutionQueue" ); - + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) .thenReturn(java.util.Optional.of(mockConfig)); NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); - + // Capture the ChildWorkflowOptions passed to newChildWorkflowStub workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( Mockito.eq(NestingExecWorkflow.class), @@ -208,40 +186,6 @@ public void testCreateProcessingWorkflowWithDynamicScalingEnabled() { Mockito.any(ChildWorkflowOptions.class)), Mockito.times(1)); } - /** - * Tests that NestingExecWorkflow uses default execution queue when config key is not present. - */ - @Test - public void testCreateProcessingWorkflowWithDynamicScalingEnabledDefaultQueue() { - // Setup - Map searchAttributes = new HashMap<>(); - WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); - - Config mockConfig = ConfigFactory.parseString( - GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=true" - ); - - workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) - .thenReturn(java.util.Optional.of(mockConfig)); - - NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); - - workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( - Mockito.eq(NestingExecWorkflow.class), - Mockito.any(ChildWorkflowOptions.class))) - .thenAnswer(invocation -> { - ChildWorkflowOptions options = invocation.getArgument(1); - // Verify task queue uses default execution queue - Assert.assertEquals(options.getTaskQueue(), - GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, - "NestingExecWorkflow should use default execution queue"); - return mockNestingWorkflow; - }); - - // Execute - workflow.createProcessingWorkflow(mockSpec, searchAttributes); - } - /** * Tests that NestingExecWorkflow does NOT have task queue set when dynamic scaling is disabled. */ @@ -250,16 +194,16 @@ public void testCreateProcessingWorkflowWithDynamicScalingDisabled() { // Setup Map searchAttributes = new HashMap<>(); WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); - + Config mockConfig = ConfigFactory.parseString( GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED + "=false" ); - + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) .thenReturn(java.util.Optional.of(mockConfig)); NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); - + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( Mockito.eq(NestingExecWorkflow.class), Mockito.any(ChildWorkflowOptions.class))) @@ -286,14 +230,14 @@ public void testCreateProcessingWorkflowWithoutDynamicScalingConfig() { // Setup Map searchAttributes = new HashMap<>(); WUProcessingSpec mockSpec = Mockito.mock(WUProcessingSpec.class); - + Config mockConfig = ConfigFactory.empty(); - + workerConfigMockedStatic.when(() -> WorkerConfig.of(Mockito.any())) .thenReturn(java.util.Optional.of(mockConfig)); NestingExecWorkflow mockNestingWorkflow = Mockito.mock(NestingExecWorkflow.class); - + workflowMockedStatic.when(() -> Workflow.newChildWorkflowStub( Mockito.eq(NestingExecWorkflow.class), Mockito.any(ChildWorkflowOptions.class))) diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java deleted file mode 100644 index 727f44b246a..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/integration/TaskQueueRoutingIntegrationTest.java +++ /dev/null @@ -1,289 +0,0 @@ -/* - * 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.temporal.integration; - -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.ddm.worker.ExecutionWorker; -import org.apache.gobblin.temporal.ddm.workflow.WorkflowStage; -import org.apache.gobblin.temporal.dynamic.ProfileOverlay; -import org.apache.gobblin.temporal.dynamic.WorkerProfile; -import org.apache.gobblin.temporal.workflows.helloworld.HelloWorldWorker; -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; - -import java.util.Arrays; -import java.util.List; - - -/** - * Integration tests verifying end-to-end task queue routing for dynamic scaling. - * Tests the complete flow from profile creation to worker configuration. - */ -public class TaskQueueRoutingIntegrationTest { - - private Config baselineConfig; - - @BeforeMethod - public void setup() { - baselineConfig = ConfigFactory.empty() - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - ConfigValueFactory.fromAnyRef(8192)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, - ConfigValueFactory.fromAnyRef(4)) - .withValue(GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE, - ConfigValueFactory.fromAnyRef("GobblinTemporalTaskQueue")) - .withValue(GobblinTemporalConfigurationKeys.EXECUTION_TASK_QUEUE, - ConfigValueFactory.fromAnyRef("GobblinTemporalExecutionQueue")) - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(HelloWorldWorker.class.getName())); - } - - /** - * Integration test: Baseline container should use default queue and WorkFulfillmentWorker. - */ - @Test - public void testBaselineContainerConfiguration() { - // Baseline container config - WorkerProfile baselineProfile = new WorkerProfile("baseline", baselineConfig); - - // Verify worker class - Assert.assertEquals( - baselineProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - HelloWorldWorker.class.getName(), - "Baseline should use HelloWorldWorker (default)"); - - // Verify task queue (would be used by AbstractTemporalWorker.getTaskQueue()) - String taskQueue = baselineProfile.getConfig().getString( - GobblinTemporalConfigurationKeys.GOBBLIN_TEMPORAL_TASK_QUEUE); - Assert.assertEquals(taskQueue, "GobblinTemporalTaskQueue", - "Baseline should use default task queue"); - } - - /** - * Integration test: Execution container should use execution queue and ExecutionWorker. - */ - @Test - public void testExecutionContainerConfiguration() { - // Create execution profile overlay - List overlayPairs = Arrays.asList( - new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ), - new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "execution" - ) - ); - - ProfileOverlay executionOverlay = new ProfileOverlay.Adding(overlayPairs); - Config executionConfig = executionOverlay.applyOverlay(baselineConfig); - WorkerProfile executionProfile = new WorkerProfile("execution", executionConfig); - - // Verify worker class - Assert.assertEquals( - executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - ExecutionWorker.class.getName(), - "Execution profile should use ExecutionWorker"); - - // Verify ExecutionWorker would use execution queue - String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(executionConfig); - Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue", - "Execution worker should use execution task queue"); - } - - /** - * Integration test: ProcessWorkUnitsWorkflow routes to execution queue when dynamic scaling enabled. - */ - @Test - public void testProcessWorkUnitsWorkflowRoutingWithDynamicScaling() { - // Setup - config with dynamic scaling enabled - Config configWithScaling = baselineConfig - .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, - ConfigValueFactory.fromAnyRef(true)); - - // Verify execution queue is configured - String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(configWithScaling); - Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue", - "ProcessWorkUnitsWorkflow should route to execution queue"); - } - - /** - * Integration test: CommitStepWorkflow always uses default queue (no explicit routing). - */ - @Test - public void testCommitStepWorkflowRoutingToDefaultQueue() { - // CommitStepWorkflow should use default queue regardless of dynamic scaling - String commitQueue = WorkflowStage.COMMIT.getTaskQueue(baselineConfig); - Assert.assertEquals(commitQueue, "GobblinTemporalTaskQueue", - "CommitStepWorkflow should use default task queue"); - - // Even with dynamic scaling enabled - Config configWithScaling = baselineConfig - .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, - ConfigValueFactory.fromAnyRef(true)); - - String commitQueueWithScaling = WorkflowStage.COMMIT.getTaskQueue(configWithScaling); - Assert.assertEquals(commitQueueWithScaling, "GobblinTemporalTaskQueue", - "CommitStepWorkflow should still use default queue with dynamic scaling"); - } - - /** - * Integration test: Complete flow from baseline to execution profile. - */ - @Test - public void testCompleteProfileDerivationFlow() { - // Step 1: Start with baseline profile - WorkerProfile baselineProfile = new WorkerProfile("baseline", baselineConfig); - Assert.assertEquals( - baselineProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - HelloWorldWorker.class.getName()); - - // Step 2: Create execution overlay - List overlayPairs = Arrays.asList( - new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ), - new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - "65536" - ) - ); - ProfileOverlay executionOverlay = new ProfileOverlay.Adding(overlayPairs); - - // Step 3: Apply overlay to create execution profile - Config executionConfig = executionOverlay.applyOverlay(baselineConfig); - WorkerProfile executionProfile = new WorkerProfile("execution-derived", executionConfig); - - // Step 4: Verify execution profile has correct settings - Assert.assertEquals( - executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - ExecutionWorker.class.getName(), - "Derived profile should have ExecutionWorker"); - Assert.assertEquals( - executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - 65536, - "Derived profile should have increased memory"); - Assert.assertEquals( - executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), - 4, - "Derived profile should preserve baseline cores"); - } - - /** - * Integration test: Verify worker class constants are correctly defined. - */ - @Test - public void testWorkerClassConstants() { - // Verify constants match actual class names - Assert.assertEquals( - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - ExecutionWorker.class.getName(), - "EXECUTION_WORKER_CLASS constant should match ExecutionWorker class name"); - - Assert.assertEquals( - GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS, - HelloWorldWorker.class.getName(), - "DEFAULT_WORKER_CLASS constant should match HelloWorldWorker class name"); - } - - /** - * Integration test: Verify task queue constants are correctly defined. - */ - @Test - public void testTaskQueueConstants() { - Assert.assertEquals( - GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE, - "GobblinTemporalExecutionQueue", - "Default execution queue constant should be correct"); - - Assert.assertEquals( - GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, - "GobblinTemporalTaskQueue", - "Default task queue constant should be correct"); - } - - /** - * Integration test: Verify different workflow stages route to correct queues. - */ - @Test - public void testWorkflowStageRouting() { - Config config = baselineConfig - .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, - ConfigValueFactory.fromAnyRef(true)); - - // Discovery activities -> default queue - String discoveryQueue = WorkflowStage.WORK_DISCOVERY.getTaskQueue(config); - Assert.assertEquals(discoveryQueue, "GobblinTemporalTaskQueue"); - - // Execution activities -> execution queue - String executionQueue = WorkflowStage.WORK_EXECUTION.getTaskQueue(config); - Assert.assertEquals(executionQueue, "GobblinTemporalExecutionQueue"); - - // Commit activities -> default queue - String commitQueue = WorkflowStage.COMMIT.getTaskQueue(config); - Assert.assertEquals(commitQueue, "GobblinTemporalTaskQueue"); - - // Verify execution uses different queue - Assert.assertNotEquals(executionQueue, discoveryQueue, - "Execution should use different queue than discovery"); - Assert.assertNotEquals(executionQueue, commitQueue, - "Execution should use different queue than commit"); - } - - /** - * Integration test: Verify memory override in execution profile. - */ - @Test - public void testExecutionProfileMemoryOverride() { - // Setup - config with stage-specific memory - Config configWithStageMemory = baselineConfig - .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, - ConfigValueFactory.fromAnyRef("32768")); - - // Create execution profile with memory override - ProfileOverlay.KVPair memoryOverride = new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - configWithStageMemory.getString(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB) - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(memoryOverride); - Config executionConfig = overlay.applyOverlay(baselineConfig); - - // Verify memory was overridden - Assert.assertEquals( - executionConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - 32768, - "Execution profile should have stage-specific memory"); - - // Verify baseline memory is different - Assert.assertEquals( - baselineConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - 8192, - "Baseline should have original memory"); - } -} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java deleted file mode 100644 index b79f8089381..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.temporal.yarn; - -import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; - -/** - * {@link DummyScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. - * This class is meant to be used for integration testing purposes only. - * This is initialized using config {@link org.apache.gobblin.yarn.GobblinYarnConfigurationKeys#APP_MASTER_SERVICE_CLASSES} while testing - */ -public class DummyDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { - - public DummyDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { - super(appMaster); - } - - @Override - protected ScalingDirectiveSource createScalingDirectiveSource() { - return new DummyScalingDirectiveSource(); - } -} \ No newline at end of file diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java deleted file mode 100644 index a2fadeb80c7..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingInitializationTest.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * 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.temporal.yarn; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.mockito.MockedStatic; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; - -import com.google.common.base.Optional; -import com.google.common.eventbus.EventBus; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.dynamic.WorkerProfile; -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyString; - - -/** - * Tests for {@link DynamicScalingYarnService} initialization and execution worker profile setup. - * Focuses on verifying that dynamic scaling correctly initializes execution worker profiles. - */ -public class DynamicScalingInitializationTest { - - private Config baseConfig; - private YarnConfiguration yarnConfiguration; - private FileSystem mockFileSystem; - private EventBus eventBus; - private AMRMClientAsync mockAMRMClient; - private RegisterApplicationMasterResponse mockRegisterResponse; - private MockedStatic amrmClientMockStatic; - - @BeforeMethod - public void setup() throws Exception { - baseConfig = ConfigFactory.empty() - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - ConfigValueFactory.fromAnyRef(8192)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, - ConfigValueFactory.fromAnyRef(4)) - .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, - ConfigValueFactory.fromAnyRef(1)) - .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, - ConfigValueFactory.fromAnyRef(true)); - - yarnConfiguration = new YarnConfiguration(); - mockFileSystem = Mockito.mock(FileSystem.class); - eventBus = new EventBus("DynamicScalingInitializationTest"); - - mockAMRMClient = Mockito.mock(AMRMClientAsync.class); - mockRegisterResponse = Mockito.mock(RegisterApplicationMasterResponse.class); - - amrmClientMockStatic = Mockito.mockStatic(AMRMClientAsync.class); - amrmClientMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) - .thenReturn(mockAMRMClient); - - Mockito.doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); - Mockito.when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) - .thenReturn(mockRegisterResponse); - Mockito.when(mockRegisterResponse.getMaximumResourceCapability()) - .thenReturn(Resource.newInstance(102400, 32)); - } - - @AfterMethod - public void tearDown() { - if (amrmClientMockStatic != null) { - amrmClientMockStatic.close(); - } - } - - /** - * Tests that DynamicScalingYarnService initializes execution worker profile when enabled. - */ - @Test - public void testDynamicScalingInitializesExecutionProfile() throws Exception { - // Execute - DynamicScalingYarnService service = new DynamicScalingYarnService( - baseConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); - DynamicScalingYarnService serviceSpy = Mockito.spy(service); - Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); - - serviceSpy.startUp(); - - // Verify that execution worker profile was requested - Mockito.verify(serviceSpy, Mockito.atLeastOnce()) - .requestContainersForWorkerProfile(any(WorkerProfile.class), anyInt()); - } - - /** - * Tests that execution worker profile has correct worker class configured. - */ - @Test - public void testExecutionProfileHasCorrectWorkerClass() { - // Setup - config with stage-specific memory - Config configWithMemory = baseConfig - .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, - ConfigValueFactory.fromAnyRef("32768")); - - // Simulate profile creation (as done in DynamicScalingYarnService) - Config profileConfig = configWithMemory - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS)) - .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - ConfigValueFactory.fromAnyRef("execution")) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - ConfigValueFactory.fromAnyRef(32768)); - - WorkerProfile executionProfile = new WorkerProfile("initial-execution", profileConfig); - - // Verify - Assert.assertEquals(executionProfile.getName(), "initial-execution"); - Assert.assertEquals( - executionProfile.getConfig().getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - "Execution profile should have ExecutionWorker class"); - Assert.assertEquals( - executionProfile.getConfig().getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - 32768, - "Execution profile should have stage-specific memory"); - } - - /** - * Tests that dynamic scaling is NOT initialized when disabled. - */ - @Test - public void testDynamicScalingNotInitializedWhenDisabled() throws Exception { - // Setup - config with dynamic scaling disabled - Config disabledConfig = baseConfig - .withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_ENABLED, - ConfigValueFactory.fromAnyRef(false)); - - // Execute - YarnService service = new YarnService( - disabledConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); - YarnService serviceSpy = Mockito.spy(service); - Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); - - serviceSpy.startUp(); - - // Verify - should request baseline containers, not execution profile - Mockito.verify(serviceSpy, Mockito.times(1)) - .requestContainers(anyInt(), any(Resource.class), any(Optional.class)); - } - - /** - * Tests that execution profile uses stage-specific memory when configured. - */ - @Test - public void testExecutionProfileUsesStageSpecificMemory() { - // Setup - int stageMemoryMb = 65536; - Config configWithStageMemory = baseConfig - .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, - ConfigValueFactory.fromAnyRef(String.valueOf(stageMemoryMb))); - - // Verify config has stage-specific memory - Assert.assertTrue(configWithStageMemory.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), - "Config should have stage-specific memory"); - Assert.assertEquals( - configWithStageMemory.getString(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), - String.valueOf(stageMemoryMb), - "Stage-specific memory should be set correctly"); - } - - /** - * Tests that execution profile falls back to baseline memory when not configured. - */ - @Test - public void testExecutionProfileFallsBackToBaselineMemory() { - // baseConfig doesn't have stage-specific memory - Assert.assertFalse(baseConfig.hasPath(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB), - "Config should not have stage-specific memory"); - - // Verify baseline memory is used - int baselineMemory = baseConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY); - Assert.assertEquals(baselineMemory, 8192, - "Should use baseline memory when stage-specific not configured"); - } - - /** - * Tests that execution profile has correct Helix tag. - */ - @Test - public void testExecutionProfileHasCorrectHelixTag() { - // Simulate profile with Helix tag - Config profileConfig = baseConfig - .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - ConfigValueFactory.fromAnyRef("execution")); - - Assert.assertEquals( - profileConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), - "execution", - "Execution profile should have 'execution' Helix tag"); - } - - /** - * Tests that initial scaling directive has correct set point. - */ - @Test - public void testInitialScalingDirectiveSetPoint() throws Exception { - // Execute - DynamicScalingYarnService service = new DynamicScalingYarnService( - baseConfig, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); - DynamicScalingYarnService serviceSpy = Mockito.spy(service); - Mockito.doNothing().when(serviceSpy).requestContainers(anyInt(), any(Resource.class), any(Optional.class)); - - serviceSpy.startUp(); - - // Verify - should request 1 execution container (set point = 1) - Mockito.verify(serviceSpy, Mockito.atLeastOnce()) - .requestContainersForWorkerProfile(any(WorkerProfile.class), Mockito.eq(1)); - } -} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java deleted file mode 100644 index 666e3c54c4b..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.temporal.yarn; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.MockitoAnnotations; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.dynamic.ScalingDirective; -import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; -import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; - -/** Tests for {@link AbstractDynamicScalingYarnServiceManager}*/ -public class DynamicScalingYarnServiceManagerTest { - - @Mock private DynamicScalingYarnService mockDynamicScalingYarnService; - @Mock private ScalingDirectiveSource mockScalingDirectiveSource; - @Mock private GobblinTemporalApplicationMaster mockGobblinTemporalApplicationMaster; - - @BeforeMethod - public void setup() { - MockitoAnnotations.openMocks(this); - // Using 1 second as polling interval so that the test runs faster and - // GetScalingDirectivesRunnable.run() will be called equal to amount of sleep introduced between startUp - // and shutDown in seconds - Config config = ConfigFactory.empty().withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_POLLING_INTERVAL_SECS, ConfigValueFactory.fromAnyRef(1)); - Mockito.when(mockGobblinTemporalApplicationMaster.getConfig()).thenReturn(config); - Mockito.when(mockGobblinTemporalApplicationMaster.get_yarnService()).thenReturn(mockDynamicScalingYarnService); - } - - @Test - public void testWhenScalingDirectivesIsNulOrEmpty() throws IOException, InterruptedException { - Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null).thenReturn(new ArrayList<>()); - TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( - mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); - testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(3000); - testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.never()).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).calcDeltasAndRequestContainers(); - } - - @Test - public void testWhenScalingDirectivesThrowsFNFE() throws IOException, InterruptedException { - Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenThrow(FileNotFoundException.class); - TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( - mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); - testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(2000); - testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.never()).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).calcDeltasAndRequestContainers(); - } - - /** Note : this test uses {@link DummyScalingDirectiveSource}*/ - @Test - public void testWithDummyScalingDirectiveSource() throws IOException, InterruptedException { - // DummyScalingDirectiveSource returns 2 scaling directives in first 5 invocations and after that it returns empty list - // so the total number of invocations after five invocations should always be 5 - TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( - mockGobblinTemporalApplicationMaster, new DummyScalingDirectiveSource()); - testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(7000); // 7 seconds sleep so that GetScalingDirectivesRunnable.run() is called for 7 times - testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(5)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).calcDeltasAndRequestContainers(); - } - - @Test - public void testWithRandomScalingDirectives() throws IOException, InterruptedException { - ScalingDirective mockScalingDirective = Mockito.mock(ScalingDirective.class); - List mockedScalingDirectives = Arrays.asList(mockScalingDirective, mockScalingDirective); - Mockito.when(mockScalingDirectiveSource.getScalingDirectives()) - .thenReturn(new ArrayList<>()) - .thenReturn(mockedScalingDirectives) - .thenReturn(mockedScalingDirectives) - .thenReturn(null); - - TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( - mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); - testDynamicScalingYarnServiceManager.startUp(); - Thread.sleep(5000); - testDynamicScalingYarnServiceManager.shutDown(); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).calcDeltasAndRequestContainers(); - } - - /** Test implementation of {@link AbstractDynamicScalingYarnServiceManager} which returns passed - * {@link ScalingDirectiveSource} when {@link #createScalingDirectiveSource()} is called while initialising - * {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable} - * */ - protected static class TestDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { - private final ScalingDirectiveSource _scalingDirectiveSource; - public TestDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster, ScalingDirectiveSource scalingDirectiveSource) { - super(appMaster); - this._scalingDirectiveSource = scalingDirectiveSource; - } - - @Override - protected ScalingDirectiveSource createScalingDirectiveSource() { - return this._scalingDirectiveSource; - } - } - -} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java deleted file mode 100644 index 9556bccd1b4..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java +++ /dev/null @@ -1,291 +0,0 @@ -/* - * 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.temporal.yarn; - -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerExitStatus; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.mockito.ArgumentCaptor; -import org.mockito.MockedStatic; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import com.google.common.base.Optional; -import com.google.common.eventbus.EventBus; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.temporal.dynamic.ScalingDirective; -import org.apache.gobblin.temporal.dynamic.WorkerProfile; -import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyString; - - -/** Tests for {@link DynamicScalingYarnService} */ -public class DynamicScalingYarnServiceTest { - private Config defaultConfigs; - private final int initNumContainers = 1; - private final int initMemoryMbs = 1024; - private final int initCores = 1; - private final Resource initResource = Resource.newInstance(initMemoryMbs, initCores); - private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); - private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); - private final EventBus eventBus = new EventBus("TemporalDynamicScalingYarnServiceTest"); - private AMRMClientAsync mockAMRMClient; - private RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; - private WorkerProfile testBaselineworkerProfile; - private DynamicScalingYarnService dynamicScalingYarnServiceSpy; - - @BeforeClass - public void setup() throws Exception { - this.defaultConfigs = ConfigFactory.empty() - .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, ConfigValueFactory.fromAnyRef(initCores)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, ConfigValueFactory.fromAnyRef(initMemoryMbs)) - .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(initNumContainers)); - - this.testBaselineworkerProfile = new WorkerProfile(this.defaultConfigs); - - mockAMRMClient = Mockito.mock(AMRMClientAsync.class); - mockRegisterApplicationMasterResponse = Mockito.mock(RegisterApplicationMasterResponse.class); - - MockedStatic amrmClientAsyncMockStatic = Mockito.mockStatic(AMRMClientAsync.class); - - amrmClientAsyncMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) - .thenReturn(mockAMRMClient); - Mockito.doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); - - Mockito.when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) - .thenReturn(mockRegisterApplicationMasterResponse); - Mockito.when(mockRegisterApplicationMasterResponse.getMaximumResourceCapability()) - .thenReturn(Mockito.mock(Resource.class)); - } - - @BeforeMethod - public void setupMethod() throws Exception { - DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); - dynamicScalingYarnServiceSpy = Mockito.spy(dynamicScalingYarnService); - Mockito.doNothing().when(dynamicScalingYarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); - dynamicScalingYarnServiceSpy.containerMap.clear(); - } - - @AfterMethod - public void cleanupMethod() { - dynamicScalingYarnServiceSpy.containerMap.clear(); - Mockito.reset(dynamicScalingYarnServiceSpy); - } - - @Test - public void testDynamicScalingYarnServiceStartupWithInitialContainers() throws Exception { - dynamicScalingYarnServiceSpy.startUp(); - ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(initNumContainers), resourceCaptor.capture(), Mockito.any(Optional.class)); - Resource capturedResource = resourceCaptor.getValue(); - Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - } - - @Test - public void testReviseWorkforcePlanAndRequestNewContainers() throws Exception { - int numNewContainers = 5; - DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); - DynamicScalingYarnService dynamicScalingYarnServiceSpy = Mockito.spy(dynamicScalingYarnService); - Mockito.doNothing().when(dynamicScalingYarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); - ScalingDirective baseScalingDirective = new ScalingDirective(WorkforceProfiles.BASELINE_NAME, numNewContainers, System.currentTimeMillis()); - dynamicScalingYarnServiceSpy.reviseWorkforcePlanAndRequestNewContainers(Collections.singletonList(baseScalingDirective)); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(numNewContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); - } - - @DataProvider(name = "OOMExitStatusProvider") - public Object[][] OOMExitStatusProvider() { - return new Object[][] { - {ContainerExitStatus.KILLED_EXCEEDED_PMEM}, - {ContainerExitStatus.KILLED_EXCEEDED_VMEM}, - {DynamicScalingYarnService.GENERAL_OOM_EXIT_STATUS_CODE} - }; - } - - @DataProvider(name = "NonOOMExitStatusProviderWhichRequestReplacementContainer") - public Object[][] NonOOMExitStatusProviderWhichRequestReplacementContainer() { - return new Object[][] { - {ContainerExitStatus.ABORTED}, - {ContainerExitStatus.PREEMPTED} - }; - } - - @DataProvider(name = "ExitStatusProviderWhichDoesNotRequestReplacementContainer") - public Object[][] ExitStatusProviderWhichDoesNotRequestReplacementContainer() { - return new Object[][] { - {ContainerExitStatus.SUCCESS}, - {ContainerExitStatus.INVALID}, - {ContainerExitStatus.DISKS_FAILED}, - {ContainerExitStatus.KILLED_BY_APPMASTER}, - {ContainerExitStatus.KILLED_BY_RESOURCEMANAGER}, - {ContainerExitStatus.KILLED_AFTER_APP_COMPLETION}, - {ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER} - }; - } - - @Test(dataProvider = "OOMExitStatusProvider") - public void testHandleContainerCompletionForStatusOOM(int containerExitStatusCode) throws Exception { - ContainerId containerId = generateRandomContainerId(); - DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); - ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); - Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); - dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(2)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(2)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); - Resource capturedResource = resourceCaptor.getValue(); - Assert.assertEquals(capturedResource.getMemorySize(), (long) initMemoryMbs * DynamicScalingYarnService.DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - } - - @Test(dataProvider = "NonOOMExitStatusProviderWhichRequestReplacementContainer") - public void testHandleContainerCompletionForNonOOMStatusWhichRequestReplacementContainer(int containerExitStatusCode) throws Exception { - ContainerId containerId = generateRandomContainerId(); - DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); - ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); - Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); - dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); - Resource capturedResource = resourceCaptor.getValue(); - Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - } - - @Test - public void testHandleContainerCompletionForAllOOMStatus() throws Exception { - ContainerId containerId1 = generateRandomContainerId(); - ContainerId containerId2 = generateRandomContainerId(); - ContainerId containerId3 = generateRandomContainerId(); - - DynamicScalingYarnService.ContainerInfo containerInfo1 = createBaselineContainerInfo(containerId1); - DynamicScalingYarnService.ContainerInfo containerInfo2 = createBaselineContainerInfo(containerId2); - DynamicScalingYarnService.ContainerInfo containerInfo3 = createBaselineContainerInfo(containerId3); - - ContainerStatus containerStatus1 = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus1.getContainerId()).thenReturn(containerId1); - Mockito.when(containerStatus1.getExitStatus()).thenReturn(ContainerExitStatus.KILLED_EXCEEDED_VMEM); - - ContainerStatus containerStatus2 = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus2.getContainerId()).thenReturn(containerId2); - Mockito.when(containerStatus2.getExitStatus()).thenReturn(DynamicScalingYarnService.GENERAL_OOM_EXIT_STATUS_CODE); - - ContainerStatus containerStatus3 = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus3.getContainerId()).thenReturn(containerId3); - Mockito.when(containerStatus3.getExitStatus()).thenReturn(ContainerExitStatus.KILLED_EXCEEDED_PMEM); - - // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated - dynamicScalingYarnServiceSpy.containerMap.put(containerId1, containerInfo1); - dynamicScalingYarnServiceSpy.containerMap.put(containerId2, containerInfo2); - dynamicScalingYarnServiceSpy.containerMap.put(containerId3, containerInfo3); - - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus1); - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus2); - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus3); - - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(4)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(4)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); - - List capturedResources = resourceCaptor.getAllValues(); - Assert.assertEquals(capturedResources.size(), 4); - - Resource capturedResource = capturedResources.get(0); - Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - - for (int idx = 1 ; idx < 4 ; idx++) { - capturedResource = capturedResources.get(idx); - Assert.assertEquals(capturedResource.getMemorySize(), (long) initMemoryMbs * DynamicScalingYarnService.DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - } - } - - @Test(dataProvider = "ExitStatusProviderWhichDoesNotRequestReplacementContainer") - public void testHandleContainerCompletionForExitStatusWhichDoesNotRequestReplacementContainer(int containerExitStatusCode) throws Exception { - ContainerId containerId = generateRandomContainerId(); - DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); - ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); - Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); - Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); - dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated - dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); - } - - @Test - public void testContainerRequestedWhenCompletionCalledBeforeAllocated() { - ContainerId containerId = generateRandomContainerId(); - DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); - dynamicScalingYarnServiceSpy.removedContainerIds.add(containerId); - dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); - dynamicScalingYarnServiceSpy.calcDeltasAndRequestContainers(); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); - ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); - Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); - Resource capturedResource = resourceCaptor.getValue(); - Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); - Assert.assertEquals(capturedResource.getVirtualCores(), initCores); - } - - - private ContainerId generateRandomContainerId() { - return ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 0), - 0), (long) (Math.random() * 1000)); - } - - private DynamicScalingYarnService.ContainerInfo createBaselineContainerInfo(ContainerId containerId) { - Container container = Container.newInstance(containerId, null, null, initResource, null, null); - return dynamicScalingYarnServiceSpy.new ContainerInfo(container, WorkforceProfiles.BASELINE_NAME_RENDERING, testBaselineworkerProfile); - } -} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java deleted file mode 100644 index fbd57c769fc..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/ExecutionWorkerProfileTest.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * 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.temporal.yarn; - -import java.util.Arrays; -import java.util.List; - -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys; -import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; -import org.apache.gobblin.temporal.dynamic.ProfileOverlay; -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; - - -/** - * Tests for ExecutionWorker profile overlay creation and configuration. - * Verifies that execution worker profiles are correctly configured with - * worker class, memory, and other settings. - */ -public class ExecutionWorkerProfileTest { - - private Config baseConfig; - - @BeforeMethod - public void setup() { - baseConfig = ConfigFactory.empty() - .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - ConfigValueFactory.fromAnyRef(8192)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, - ConfigValueFactory.fromAnyRef(4)); - } - - /** - * Tests that execution profile overlay sets ExecutionWorker class. - */ - @Test - public void testExecutionProfileSetsWorkerClass() { - // Create overlay pairs as DynamicScalingYarnService does - ProfileOverlay.KVPair workerClassPair = new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(workerClassPair); - Config overlaidConfig = overlay.applyOverlay(baseConfig); - - // Verify - Assert.assertTrue(overlaidConfig.hasPath(GobblinTemporalConfigurationKeys.WORKER_CLASS), - "Overlaid config should have worker class"); - Assert.assertEquals(overlaidConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - "Should set ExecutionWorker class"); - } - - /** - * Tests that execution profile overlay sets Helix tag. - */ - @Test - public void testExecutionProfileSetsHelixTag() { - // Create overlay - ProfileOverlay.KVPair helixTagPair = new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "execution" - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(helixTagPair); - Config overlaidConfig = overlay.applyOverlay(baseConfig); - - // Verify - Assert.assertTrue(overlaidConfig.hasPath(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), - "Overlaid config should have Helix tag"); - Assert.assertEquals(overlaidConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), - "execution", - "Should set execution tag"); - } - - /** - * Tests that execution profile overlay can override memory configuration. - */ - @Test - public void testExecutionProfileOverridesMemory() { - // Setup - config with stage-specific memory - Config configWithStageMemory = baseConfig - .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, - ConfigValueFactory.fromAnyRef("65536")); - - // Create overlay with memory override - ProfileOverlay.KVPair memoryPair = new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - "65536" - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(memoryPair); - Config overlaidConfig = overlay.applyOverlay(baseConfig); - - // Verify - Assert.assertEquals(overlaidConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - "65536", - "Should override memory to 64GB for execution workers"); - } - - /** - * Tests complete execution profile overlay with all settings. - */ - @Test - public void testCompleteExecutionProfileOverlay() { - // Setup - simulate DynamicScalingYarnService.createExecutionProfileOverlay() - Config configWithStageMemory = baseConfig - .withValue(GobblinTemporalConfigurationKeys.WORK_EXECUTION_MEMORY_MB, - ConfigValueFactory.fromAnyRef("32768")); - - // Create complete overlay - List overlayPairs = Arrays.asList( - new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ), - new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "execution" - ), - new ProfileOverlay.KVPair( - GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, - "32768" - ) - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(overlayPairs); - Config overlaidConfig = overlay.applyOverlay(baseConfig); - - // Verify all settings - Assert.assertEquals(overlaidConfig.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - "Should set ExecutionWorker class"); - Assert.assertEquals(overlaidConfig.getString(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY), - "execution", - "Should set execution tag"); - Assert.assertEquals(overlaidConfig.getString(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - "32768", - "Should set execution memory"); - Assert.assertEquals(overlaidConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY), - 4, - "Should preserve baseline cores"); - } - - /** - * Tests that execution profile falls back to baseline memory when stage-specific not configured. - */ - @Test - public void testExecutionProfileFallsBackToBaselineMemory() { - // Create overlay without memory (simulating no stage-specific memory config) - List overlayPairs = Arrays.asList( - new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ), - new ProfileOverlay.KVPair( - GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_KEY, - "execution" - ) - ); - - ProfileOverlay overlay = new ProfileOverlay.Adding(overlayPairs); - Config overlaidConfig = overlay.applyOverlay(baseConfig); - - // Verify baseline memory is preserved - Assert.assertEquals(overlaidConfig.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY), - 8192, - "Should use baseline memory when stage-specific not configured"); - } - - /** - * Tests that baseline profile does NOT have ExecutionWorker class. - */ - @Test - public void testBaselineProfileDoesNotHaveExecutionWorker() { - // Baseline config should not have execution worker class - Config baselineWithWorker = baseConfig - .withValue(GobblinTemporalConfigurationKeys.WORKER_CLASS, - ConfigValueFactory.fromAnyRef(GobblinTemporalConfigurationKeys.DEFAULT_WORKER_CLASS)); - - Assert.assertNotEquals( - baselineWithWorker.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS, - "Baseline profile should not use ExecutionWorker"); - } - - /** - * Tests that execution profile overlay is idempotent. - */ - @Test - public void testExecutionProfileOverlayIsIdempotent() { - // Create overlay - ProfileOverlay.KVPair workerClassPair = new ProfileOverlay.KVPair( - GobblinTemporalConfigurationKeys.WORKER_CLASS, - GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS - ); - ProfileOverlay overlay = new ProfileOverlay.Adding(workerClassPair); - - // Apply twice - Config overlaidOnce = overlay.applyOverlay(baseConfig); - Config overlaidTwice = overlay.applyOverlay(overlaidOnce); - - // Verify same result - Assert.assertEquals( - overlaidOnce.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - overlaidTwice.getString(GobblinTemporalConfigurationKeys.WORKER_CLASS), - "Overlay should be idempotent"); - } -} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java deleted file mode 100644 index 8d216450a34..00000000000 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.temporal.yarn; - -import java.io.IOException; -import java.net.URL; - -import org.apache.gobblin.temporal.dynamic.WorkerProfile; -import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -import com.google.common.base.Optional; - -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.mockito.MockedStatic; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigValueFactory; -import com.google.common.eventbus.EventBus; - -import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; - -import static org.mockito.Mockito.*; - - -/** - * Tests for {@link YarnService} - * - * NOTE : This test is a partial clone of {@link org.apache.gobblin.yarn.YarnServiceTest} - * */ -public class YarnServiceTest { - private Config defaultConfigs; - private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); - private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); - private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); - private AMRMClientAsync mockAMRMClient; - private RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; - - @BeforeClass - public void setup() throws IOException, YarnException { - mockAMRMClient = mock(AMRMClientAsync.class); - mockRegisterApplicationMasterResponse = mock(RegisterApplicationMasterResponse.class); - - URL url = YarnServiceTest.class.getClassLoader() - .getResource(YarnServiceTest.class.getSimpleName() + ".conf"); - Assert.assertNotNull(url, "Could not find resource " + url); - this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); - - MockedStatic amrmClientAsyncMockStatic = mockStatic(AMRMClientAsync.class); - - amrmClientAsyncMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) - .thenReturn(mockAMRMClient); - doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); - - when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) - .thenReturn(mockRegisterApplicationMasterResponse); - when(mockRegisterApplicationMasterResponse.getMaximumResourceCapability()) - .thenReturn(Mockito.mock(Resource.class)); - } - - @Test - public void testYarnServiceStartupWithInitialContainers() throws Exception { - int expectedNumContainers = 3; - Config config = this.defaultConfigs.withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(expectedNumContainers)); - YarnService yarnService = new YarnService(config, "testApplicationName", "testApplicationId", yarnConfiguration, mockFileSystem, eventBus); - YarnService yarnServiceSpy = Mockito.spy(yarnService); - Mockito.doNothing().when(yarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); - yarnServiceSpy.startUp(); - Mockito.verify(yarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(expectedNumContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); - } - - @Test - public void testBuildContainerCommand() throws Exception { - final double jvmMemoryXmxRatio = 0.7; - final int jvmMemoryOverheadMbs = 50; - final int resourceMemoryMB = 3072; - final int expectedJvmMemory = (int) (resourceMemoryMB * jvmMemoryXmxRatio) - jvmMemoryOverheadMbs; - - Config config = this.defaultConfigs - .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryXmxRatio)) - .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryOverheadMbs)); - - YarnService yarnService = new YarnService( - config, - "testApplicationName", - "testApplicationId", - yarnConfiguration, - mockFileSystem, - eventBus - ); - - WorkerProfile workerProfile = new WorkerProfile(config); - ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 0), - 0), 0); - Resource resource = Resource.newInstance(resourceMemoryMB, 2); - Container container = Container.newInstance(containerId, null, null, resource, null, null); - YarnService.ContainerInfo containerInfo = yarnService.new ContainerInfo(container, WorkforceProfiles.BASELINE_NAME_RENDERING, workerProfile); - String command = containerInfo.getStartupCommand(); - Assert.assertTrue(command.contains("-Xmx" + expectedJvmMemory + "M")); - } -} From 82b4dc0cac7fd149e1e653eb0d6943f6bae91720 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Tue, 16 Dec 2025 11:42:39 +0530 Subject: [PATCH 11/13] removed redundant unit tests --- .../temporal/dynamic/WorkforcePlan.java | 1 - ...DummyDynamicScalingYarnServiceManager.java | 38 +++ .../DynamicScalingYarnServiceManagerTest.java | 133 ++++++++ .../yarn/DynamicScalingYarnServiceTest.java | 291 ++++++++++++++++++ .../temporal/yarn/YarnServiceTest.java | 129 ++++++++ 5 files changed, 591 insertions(+), 1 deletion(-) create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java create mode 100644 gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java index b08ad51f722..336d357f237 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/dynamic/WorkforcePlan.java @@ -174,5 +174,4 @@ WorkerProfile peepProfile(String profileName) throws WorkforceProfiles.UnknownPr WorkerProfile peepBaselineProfile() throws WorkforceProfiles.UnknownProfileException { return profiles.getOrThrow(WorkforceProfiles.BASELINE_NAME); } - } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java new file mode 100644 index 00000000000..b79f8089381 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DummyDynamicScalingYarnServiceManager.java @@ -0,0 +1,38 @@ +/* + * 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.temporal.yarn; + +import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; + +/** + * {@link DummyScalingDirectiveSource} based implementation of {@link AbstractDynamicScalingYarnServiceManager}. + * This class is meant to be used for integration testing purposes only. + * This is initialized using config {@link org.apache.gobblin.yarn.GobblinYarnConfigurationKeys#APP_MASTER_SERVICE_CLASSES} while testing + */ +public class DummyDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + + public DummyDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster) { + super(appMaster); + } + + @Override + protected ScalingDirectiveSource createScalingDirectiveSource() { + return new DummyScalingDirectiveSource(); + } +} \ No newline at end of file diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java new file mode 100644 index 00000000000..666e3c54c4b --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceManagerTest.java @@ -0,0 +1,133 @@ +/* + * 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.temporal.yarn; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys; +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.ScalingDirectiveSource; +import org.apache.gobblin.temporal.dynamic.DummyScalingDirectiveSource; + +/** Tests for {@link AbstractDynamicScalingYarnServiceManager}*/ +public class DynamicScalingYarnServiceManagerTest { + + @Mock private DynamicScalingYarnService mockDynamicScalingYarnService; + @Mock private ScalingDirectiveSource mockScalingDirectiveSource; + @Mock private GobblinTemporalApplicationMaster mockGobblinTemporalApplicationMaster; + + @BeforeMethod + public void setup() { + MockitoAnnotations.openMocks(this); + // Using 1 second as polling interval so that the test runs faster and + // GetScalingDirectivesRunnable.run() will be called equal to amount of sleep introduced between startUp + // and shutDown in seconds + Config config = ConfigFactory.empty().withValue(GobblinTemporalConfigurationKeys.DYNAMIC_SCALING_POLLING_INTERVAL_SECS, ConfigValueFactory.fromAnyRef(1)); + Mockito.when(mockGobblinTemporalApplicationMaster.getConfig()).thenReturn(config); + Mockito.when(mockGobblinTemporalApplicationMaster.get_yarnService()).thenReturn(mockDynamicScalingYarnService); + } + + @Test + public void testWhenScalingDirectivesIsNulOrEmpty() throws IOException, InterruptedException { + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenReturn(null).thenReturn(new ArrayList<>()); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(3000); + testDynamicScalingYarnServiceManager.shutDown(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.never()).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).calcDeltasAndRequestContainers(); + } + + @Test + public void testWhenScalingDirectivesThrowsFNFE() throws IOException, InterruptedException { + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()).thenThrow(FileNotFoundException.class); + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(2000); + testDynamicScalingYarnServiceManager.shutDown(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.never()).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).calcDeltasAndRequestContainers(); + } + + /** Note : this test uses {@link DummyScalingDirectiveSource}*/ + @Test + public void testWithDummyScalingDirectiveSource() throws IOException, InterruptedException { + // DummyScalingDirectiveSource returns 2 scaling directives in first 5 invocations and after that it returns empty list + // so the total number of invocations after five invocations should always be 5 + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, new DummyScalingDirectiveSource()); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(7000); // 7 seconds sleep so that GetScalingDirectivesRunnable.run() is called for 7 times + testDynamicScalingYarnServiceManager.shutDown(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(5)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).calcDeltasAndRequestContainers(); + } + + @Test + public void testWithRandomScalingDirectives() throws IOException, InterruptedException { + ScalingDirective mockScalingDirective = Mockito.mock(ScalingDirective.class); + List mockedScalingDirectives = Arrays.asList(mockScalingDirective, mockScalingDirective); + Mockito.when(mockScalingDirectiveSource.getScalingDirectives()) + .thenReturn(new ArrayList<>()) + .thenReturn(mockedScalingDirectives) + .thenReturn(mockedScalingDirectives) + .thenReturn(null); + + TestDynamicScalingYarnServiceManager testDynamicScalingYarnServiceManager = new TestDynamicScalingYarnServiceManager( + mockGobblinTemporalApplicationMaster, mockScalingDirectiveSource); + testDynamicScalingYarnServiceManager.startUp(); + Thread.sleep(5000); + testDynamicScalingYarnServiceManager.shutDown(); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(2)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(mockDynamicScalingYarnService, Mockito.times(3)).calcDeltasAndRequestContainers(); + } + + /** Test implementation of {@link AbstractDynamicScalingYarnServiceManager} which returns passed + * {@link ScalingDirectiveSource} when {@link #createScalingDirectiveSource()} is called while initialising + * {@link AbstractDynamicScalingYarnServiceManager.GetScalingDirectivesRunnable} + * */ + protected static class TestDynamicScalingYarnServiceManager extends AbstractDynamicScalingYarnServiceManager { + private final ScalingDirectiveSource _scalingDirectiveSource; + public TestDynamicScalingYarnServiceManager(GobblinTemporalApplicationMaster appMaster, ScalingDirectiveSource scalingDirectiveSource) { + super(appMaster); + this._scalingDirectiveSource = scalingDirectiveSource; + } + + @Override + protected ScalingDirectiveSource createScalingDirectiveSource() { + return this._scalingDirectiveSource; + } + } + +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java new file mode 100644 index 00000000000..9556bccd1b4 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/DynamicScalingYarnServiceTest.java @@ -0,0 +1,291 @@ +/* + * 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.temporal.yarn; + +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.mockito.ArgumentCaptor; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import com.google.common.base.Optional; +import com.google.common.eventbus.EventBus; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.temporal.dynamic.ScalingDirective; +import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; + + +/** Tests for {@link DynamicScalingYarnService} */ +public class DynamicScalingYarnServiceTest { + private Config defaultConfigs; + private final int initNumContainers = 1; + private final int initMemoryMbs = 1024; + private final int initCores = 1; + private final Resource initResource = Resource.newInstance(initMemoryMbs, initCores); + private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); + private final EventBus eventBus = new EventBus("TemporalDynamicScalingYarnServiceTest"); + private AMRMClientAsync mockAMRMClient; + private RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; + private WorkerProfile testBaselineworkerProfile; + private DynamicScalingYarnService dynamicScalingYarnServiceSpy; + + @BeforeClass + public void setup() throws Exception { + this.defaultConfigs = ConfigFactory.empty() + .withValue(GobblinYarnConfigurationKeys.CONTAINER_CORES_KEY, ConfigValueFactory.fromAnyRef(initCores)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY, ConfigValueFactory.fromAnyRef(initMemoryMbs)) + .withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(initNumContainers)); + + this.testBaselineworkerProfile = new WorkerProfile(this.defaultConfigs); + + mockAMRMClient = Mockito.mock(AMRMClientAsync.class); + mockRegisterApplicationMasterResponse = Mockito.mock(RegisterApplicationMasterResponse.class); + + MockedStatic amrmClientAsyncMockStatic = Mockito.mockStatic(AMRMClientAsync.class); + + amrmClientAsyncMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) + .thenReturn(mockAMRMClient); + Mockito.doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); + + Mockito.when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) + .thenReturn(mockRegisterApplicationMasterResponse); + Mockito.when(mockRegisterApplicationMasterResponse.getMaximumResourceCapability()) + .thenReturn(Mockito.mock(Resource.class)); + } + + @BeforeMethod + public void setupMethod() throws Exception { + DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + dynamicScalingYarnServiceSpy = Mockito.spy(dynamicScalingYarnService); + Mockito.doNothing().when(dynamicScalingYarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); + dynamicScalingYarnServiceSpy.containerMap.clear(); + } + + @AfterMethod + public void cleanupMethod() { + dynamicScalingYarnServiceSpy.containerMap.clear(); + Mockito.reset(dynamicScalingYarnServiceSpy); + } + + @Test + public void testDynamicScalingYarnServiceStartupWithInitialContainers() throws Exception { + dynamicScalingYarnServiceSpy.startUp(); + ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(initNumContainers), resourceCaptor.capture(), Mockito.any(Optional.class)); + Resource capturedResource = resourceCaptor.getValue(); + Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + } + + @Test + public void testReviseWorkforcePlanAndRequestNewContainers() throws Exception { + int numNewContainers = 5; + DynamicScalingYarnService dynamicScalingYarnService = new DynamicScalingYarnService(this.defaultConfigs, "testApp", "testAppId", yarnConfiguration, mockFileSystem, eventBus); + DynamicScalingYarnService dynamicScalingYarnServiceSpy = Mockito.spy(dynamicScalingYarnService); + Mockito.doNothing().when(dynamicScalingYarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); + ScalingDirective baseScalingDirective = new ScalingDirective(WorkforceProfiles.BASELINE_NAME, numNewContainers, System.currentTimeMillis()); + dynamicScalingYarnServiceSpy.reviseWorkforcePlanAndRequestNewContainers(Collections.singletonList(baseScalingDirective)); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(numNewContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); + } + + @DataProvider(name = "OOMExitStatusProvider") + public Object[][] OOMExitStatusProvider() { + return new Object[][] { + {ContainerExitStatus.KILLED_EXCEEDED_PMEM}, + {ContainerExitStatus.KILLED_EXCEEDED_VMEM}, + {DynamicScalingYarnService.GENERAL_OOM_EXIT_STATUS_CODE} + }; + } + + @DataProvider(name = "NonOOMExitStatusProviderWhichRequestReplacementContainer") + public Object[][] NonOOMExitStatusProviderWhichRequestReplacementContainer() { + return new Object[][] { + {ContainerExitStatus.ABORTED}, + {ContainerExitStatus.PREEMPTED} + }; + } + + @DataProvider(name = "ExitStatusProviderWhichDoesNotRequestReplacementContainer") + public Object[][] ExitStatusProviderWhichDoesNotRequestReplacementContainer() { + return new Object[][] { + {ContainerExitStatus.SUCCESS}, + {ContainerExitStatus.INVALID}, + {ContainerExitStatus.DISKS_FAILED}, + {ContainerExitStatus.KILLED_BY_APPMASTER}, + {ContainerExitStatus.KILLED_BY_RESOURCEMANAGER}, + {ContainerExitStatus.KILLED_AFTER_APP_COMPLETION}, + {ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER} + }; + } + + @Test(dataProvider = "OOMExitStatusProvider") + public void testHandleContainerCompletionForStatusOOM(int containerExitStatusCode) throws Exception { + ContainerId containerId = generateRandomContainerId(); + DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); + ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); + Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); + dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(2)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(2)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); + Resource capturedResource = resourceCaptor.getValue(); + Assert.assertEquals(capturedResource.getMemorySize(), (long) initMemoryMbs * DynamicScalingYarnService.DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + } + + @Test(dataProvider = "NonOOMExitStatusProviderWhichRequestReplacementContainer") + public void testHandleContainerCompletionForNonOOMStatusWhichRequestReplacementContainer(int containerExitStatusCode) throws Exception { + ContainerId containerId = generateRandomContainerId(); + DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); + ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); + Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); + dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); + Resource capturedResource = resourceCaptor.getValue(); + Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + } + + @Test + public void testHandleContainerCompletionForAllOOMStatus() throws Exception { + ContainerId containerId1 = generateRandomContainerId(); + ContainerId containerId2 = generateRandomContainerId(); + ContainerId containerId3 = generateRandomContainerId(); + + DynamicScalingYarnService.ContainerInfo containerInfo1 = createBaselineContainerInfo(containerId1); + DynamicScalingYarnService.ContainerInfo containerInfo2 = createBaselineContainerInfo(containerId2); + DynamicScalingYarnService.ContainerInfo containerInfo3 = createBaselineContainerInfo(containerId3); + + ContainerStatus containerStatus1 = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus1.getContainerId()).thenReturn(containerId1); + Mockito.when(containerStatus1.getExitStatus()).thenReturn(ContainerExitStatus.KILLED_EXCEEDED_VMEM); + + ContainerStatus containerStatus2 = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus2.getContainerId()).thenReturn(containerId2); + Mockito.when(containerStatus2.getExitStatus()).thenReturn(DynamicScalingYarnService.GENERAL_OOM_EXIT_STATUS_CODE); + + ContainerStatus containerStatus3 = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus3.getContainerId()).thenReturn(containerId3); + Mockito.when(containerStatus3.getExitStatus()).thenReturn(ContainerExitStatus.KILLED_EXCEEDED_PMEM); + + // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated + dynamicScalingYarnServiceSpy.containerMap.put(containerId1, containerInfo1); + dynamicScalingYarnServiceSpy.containerMap.put(containerId2, containerInfo2); + dynamicScalingYarnServiceSpy.containerMap.put(containerId3, containerInfo3); + + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus1); + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus2); + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus3); + + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(3)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(4)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(4)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); + + List capturedResources = resourceCaptor.getAllValues(); + Assert.assertEquals(capturedResources.size(), 4); + + Resource capturedResource = capturedResources.get(0); + Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + + for (int idx = 1 ; idx < 4 ; idx++) { + capturedResource = capturedResources.get(idx); + Assert.assertEquals(capturedResource.getMemorySize(), (long) initMemoryMbs * DynamicScalingYarnService.DEFAULT_REPLACEMENT_CONTAINER_MEMORY_MULTIPLIER); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + } + } + + @Test(dataProvider = "ExitStatusProviderWhichDoesNotRequestReplacementContainer") + public void testHandleContainerCompletionForExitStatusWhichDoesNotRequestReplacementContainer(int containerExitStatusCode) throws Exception { + ContainerId containerId = generateRandomContainerId(); + DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); + ContainerStatus containerStatus = Mockito.mock(ContainerStatus.class); + Mockito.when(containerStatus.getContainerId()).thenReturn(containerId); + Mockito.when(containerStatus.getExitStatus()).thenReturn(containerExitStatusCode); + dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); // Required to be done for test otherwise containerMap is always empty since it is updated after containers are allocated + dynamicScalingYarnServiceSpy.handleContainerCompletion(containerStatus); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).reviseWorkforcePlanAndRequestNewContainers(Mockito.anyList()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(0)).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); + } + + @Test + public void testContainerRequestedWhenCompletionCalledBeforeAllocated() { + ContainerId containerId = generateRandomContainerId(); + DynamicScalingYarnService.ContainerInfo containerInfo = createBaselineContainerInfo(containerId); + dynamicScalingYarnServiceSpy.removedContainerIds.add(containerId); + dynamicScalingYarnServiceSpy.containerMap.put(containerId, containerInfo); + dynamicScalingYarnServiceSpy.calcDeltasAndRequestContainers(); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainersForWorkerProfile(Mockito.any(WorkerProfile.class), Mockito.anyInt()); + ArgumentCaptor resourceCaptor = ArgumentCaptor.forClass(Resource.class); + Mockito.verify(dynamicScalingYarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(1), resourceCaptor.capture(), Mockito.any(Optional.class)); + Resource capturedResource = resourceCaptor.getValue(); + Assert.assertEquals(capturedResource.getMemorySize(), initMemoryMbs); + Assert.assertEquals(capturedResource.getVirtualCores(), initCores); + } + + + private ContainerId generateRandomContainerId() { + return ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 0), + 0), (long) (Math.random() * 1000)); + } + + private DynamicScalingYarnService.ContainerInfo createBaselineContainerInfo(ContainerId containerId) { + Container container = Container.newInstance(containerId, null, null, initResource, null, null); + return dynamicScalingYarnServiceSpy.new ContainerInfo(container, WorkforceProfiles.BASELINE_NAME_RENDERING, testBaselineworkerProfile); + } +} diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java new file mode 100644 index 00000000000..8d216450a34 --- /dev/null +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/yarn/YarnServiceTest.java @@ -0,0 +1,129 @@ +/* + * 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.temporal.yarn; + +import java.io.IOException; +import java.net.URL; + +import org.apache.gobblin.temporal.dynamic.WorkerProfile; +import org.apache.gobblin.temporal.dynamic.WorkforceProfiles; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import com.google.common.base.Optional; + +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; +import com.google.common.eventbus.EventBus; + +import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys; + +import static org.mockito.Mockito.*; + + +/** + * Tests for {@link YarnService} + * + * NOTE : This test is a partial clone of {@link org.apache.gobblin.yarn.YarnServiceTest} + * */ +public class YarnServiceTest { + private Config defaultConfigs; + private final YarnConfiguration yarnConfiguration = new YarnConfiguration(); + private final FileSystem mockFileSystem = Mockito.mock(FileSystem.class); + private final EventBus eventBus = new EventBus("TemporalYarnServiceTest"); + private AMRMClientAsync mockAMRMClient; + private RegisterApplicationMasterResponse mockRegisterApplicationMasterResponse; + + @BeforeClass + public void setup() throws IOException, YarnException { + mockAMRMClient = mock(AMRMClientAsync.class); + mockRegisterApplicationMasterResponse = mock(RegisterApplicationMasterResponse.class); + + URL url = YarnServiceTest.class.getClassLoader() + .getResource(YarnServiceTest.class.getSimpleName() + ".conf"); + Assert.assertNotNull(url, "Could not find resource " + url); + this.defaultConfigs = ConfigFactory.parseURL(url).resolve(); + + MockedStatic amrmClientAsyncMockStatic = mockStatic(AMRMClientAsync.class); + + amrmClientAsyncMockStatic.when(() -> AMRMClientAsync.createAMRMClientAsync(anyInt(), any(AMRMClientAsync.CallbackHandler.class))) + .thenReturn(mockAMRMClient); + doNothing().when(mockAMRMClient).init(any(YarnConfiguration.class)); + + when(mockAMRMClient.registerApplicationMaster(anyString(), anyInt(), anyString())) + .thenReturn(mockRegisterApplicationMasterResponse); + when(mockRegisterApplicationMasterResponse.getMaximumResourceCapability()) + .thenReturn(Mockito.mock(Resource.class)); + } + + @Test + public void testYarnServiceStartupWithInitialContainers() throws Exception { + int expectedNumContainers = 3; + Config config = this.defaultConfigs.withValue(GobblinYarnConfigurationKeys.INITIAL_CONTAINERS_KEY, ConfigValueFactory.fromAnyRef(expectedNumContainers)); + YarnService yarnService = new YarnService(config, "testApplicationName", "testApplicationId", yarnConfiguration, mockFileSystem, eventBus); + YarnService yarnServiceSpy = Mockito.spy(yarnService); + Mockito.doNothing().when(yarnServiceSpy).requestContainers(Mockito.anyInt(), Mockito.any(Resource.class), Mockito.any(Optional.class)); + yarnServiceSpy.startUp(); + Mockito.verify(yarnServiceSpy, Mockito.times(1)).requestContainers(Mockito.eq(expectedNumContainers), Mockito.any(Resource.class), Mockito.any(Optional.class)); + } + + @Test + public void testBuildContainerCommand() throws Exception { + final double jvmMemoryXmxRatio = 0.7; + final int jvmMemoryOverheadMbs = 50; + final int resourceMemoryMB = 3072; + final int expectedJvmMemory = (int) (resourceMemoryMB * jvmMemoryXmxRatio) - jvmMemoryOverheadMbs; + + Config config = this.defaultConfigs + .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_XMX_RATIO_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryXmxRatio)) + .withValue(GobblinYarnConfigurationKeys.CONTAINER_JVM_MEMORY_OVERHEAD_MBS_KEY, ConfigValueFactory.fromAnyRef(jvmMemoryOverheadMbs)); + + YarnService yarnService = new YarnService( + config, + "testApplicationName", + "testApplicationId", + yarnConfiguration, + mockFileSystem, + eventBus + ); + + WorkerProfile workerProfile = new WorkerProfile(config); + ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 0), + 0), 0); + Resource resource = Resource.newInstance(resourceMemoryMB, 2); + Container container = Container.newInstance(containerId, null, null, resource, null, null); + YarnService.ContainerInfo containerInfo = yarnService.new ContainerInfo(container, WorkforceProfiles.BASELINE_NAME_RENDERING, workerProfile); + String command = containerInfo.getStartupCommand(); + Assert.assertTrue(command.contains("-Xmx" + expectedJvmMemory + "M")); + } +} From 3528f88a9bc74363f6cadaff80773c74f41d675d Mon Sep 17 00:00:00 2001 From: agam-99 Date: Tue, 6 Jan 2026 10:43:49 +0530 Subject: [PATCH 12/13] copilot review fixes --- .../temporal/GobblinTemporalConfigurationKeys.java | 3 +++ .../gobblin/temporal/ddm/activity/ActivityType.java | 13 ------------- .../AbstractRecommendScalingForWorkUnitsImpl.java | 2 +- .../temporal/ddm/workflow/WorkflowStageTest.java | 2 +- 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java index 14155cc1c23..eb794de5712 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/GobblinTemporalConfigurationKeys.java @@ -141,5 +141,8 @@ public interface GobblinTemporalConfigurationKeys { String TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = TEMPORAL_ACTIVITY_RETRY_OPTIONS + "maximum.attempts"; int DEFAULT_TEMPORAL_ACTIVITY_RETRY_OPTIONS_MAXIMUM_ATTEMPTS = 4; + /** + * Memory allocation for execution worker containers. + */ String WORK_EXECUTION_MEMORY_MB = STAGE_SPECIFIC_PREFIX + "workExecution.memory.mb"; } diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java index a283c179d41..22126b2d0fc 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/ActivityType.java @@ -72,19 +72,6 @@ public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbe .build(); } - public ActivityOptions buildActivityOptions(Properties props, boolean setHeartbeatTimeout, String taskQueue) { - ActivityOptions.Builder builder = ActivityOptions.newBuilder() - .setStartToCloseTimeout(getStartToCloseTimeout(props)) - .setRetryOptions(buildRetryOptions(props)) - .setTaskQueue(taskQueue); - - if (setHeartbeatTimeout) { - builder.setHeartbeatTimeout(getHeartbeatTimeout(props)); - } - - return builder.build(); - } - private ActivityOptions buildActivityOptionsWithoutHeartBeatTimeout(Properties props) { return ActivityOptions.newBuilder() .setStartToCloseTimeout(getStartToCloseTimeout(props)) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java index 78363917ab0..aa86c6425c7 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/activity/impl/AbstractRecommendScalingForWorkUnitsImpl.java @@ -97,7 +97,7 @@ private ProfileOverlay createExecutionWorkerOverlay(JobState jobState) { GobblinTemporalConfigurationKeys.EXECUTION_WORKER_CLASS )); - return overlayPairs.isEmpty() ? ProfileOverlay.unchanged() : new ProfileOverlay.Adding(overlayPairs); + return new ProfileOverlay.Adding(overlayPairs); } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java index 7c386caf169..1a56e051314 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/workflow/WorkflowStageTest.java @@ -97,7 +97,7 @@ public void testWorkCommitStageUsesDefaultQueue() { // Verify Assert.assertEquals(taskQueue, GobblinTemporalConfigurationKeys.DEFAULT_GOBBLIN_TEMPORAL_TASK_QUEUE, - "WORK_COMMIT should use default task queue"); + "COMMIT should use default task queue"); } /** From 3fad0e6ab235ce4c43ac5335c1cc9677d99366a5 Mon Sep 17 00:00:00 2001 From: agam-99 Date: Thu, 8 Jan 2026 10:52:59 +0530 Subject: [PATCH 13/13] PR review fixes --- .../gobblin/temporal/ddm/worker/ExecutionWorker.java | 9 ++++++++- .../gobblin/temporal/ddm/worker/ExecutionWorkerTest.java | 4 ++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java index 3a2e571f033..1b36f76fef6 100644 --- a/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java +++ b/gobblin-temporal/src/main/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorker.java @@ -42,7 +42,7 @@ */ public class ExecutionWorker extends AbstractTemporalWorker { public static final long DEADLOCK_DETECTION_TIMEOUT_SECONDS = 120; - public int maxExecutionConcurrency; + private final int maxExecutionConcurrency; public ExecutionWorker(Config config, WorkflowClient workflowClient) { super(config, workflowClient); @@ -83,4 +83,11 @@ protected String getTaskQueue() { GobblinTemporalConfigurationKeys.DEFAULT_EXECUTION_TASK_QUEUE ); } + + /** + * Package-private for testing purposes. + */ + int getMaxExecutionConcurrency() { + return maxExecutionConcurrency; + } } diff --git a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java index 7d235b3c9df..c3b6234c422 100644 --- a/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java +++ b/gobblin-temporal/src/test/java/org/apache/gobblin/temporal/ddm/worker/ExecutionWorkerTest.java @@ -184,7 +184,7 @@ public void testMaxExecutionConcurrencyInitialization() throws Exception { ExecutionWorker worker = createMockWorker(config); - Assert.assertEquals(worker.maxExecutionConcurrency, customConcurrency, + Assert.assertEquals(worker.getMaxExecutionConcurrency(), customConcurrency, "maxExecutionConcurrency should be initialized from config"); } @@ -197,7 +197,7 @@ public void testMaxExecutionConcurrencyDefault() throws Exception { ExecutionWorker worker = createMockWorker(config); - Assert.assertEquals(worker.maxExecutionConcurrency, + Assert.assertEquals(worker.getMaxExecutionConcurrency(), GobblinTemporalConfigurationKeys.DEFAULT_TEMPORAL_NUM_THREADS_PER_WORKER, "maxExecutionConcurrency should use default value when not configured"); }