diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index b0fbfa4ab..caf85fb81 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -33,12 +33,6 @@ jobs: USE_DOCKER_SERVICE: false run: ./gradlew --no-daemon test -x checkLicenseMain -x checkLicenses -x spotlessCheck -x spotlessApply -x spotlessJava -P edgeDepsTest - - name: Run independent resource tuner test - env: - USER: unittest - USE_DOCKER_SERVICE: false - run: ./gradlew --no-daemon temporal-sdk:testResourceIndependent -x checkLicenseMain -x checkLicenses -x spotlessCheck -x spotlessApply -x spotlessJava -P edgeDepsTest - - name: Publish Test Report uses: mikepenz/action-junit-report@v4 if: success() || failure() # always run even if the previous step fails diff --git a/temporal-sdk/build.gradle b/temporal-sdk/build.gradle index 4bd3fdddc..e8371f29b 100644 --- a/temporal-sdk/build.gradle +++ b/temporal-sdk/build.gradle @@ -36,16 +36,3 @@ task registerNamespace(type: JavaExec) { } test.dependsOn 'registerNamespace' - -test { - useJUnit { - excludeCategories 'io.temporal.worker.IndependentResourceBasedTests' - } -} - -task testResourceIndependent(type: Test) { - useJUnit { - includeCategories 'io.temporal.worker.IndependentResourceBasedTests' - maxParallelForks = 1 - } -} \ No newline at end of file diff --git a/temporal-sdk/src/main/java/io/temporal/internal/activity/ActivityPollResponseToInfo.java b/temporal-sdk/src/main/java/io/temporal/internal/activity/ActivityPollResponseToInfo.java deleted file mode 100644 index 9af4ae9d6..000000000 --- a/temporal-sdk/src/main/java/io/temporal/internal/activity/ActivityPollResponseToInfo.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.internal.activity; - -import io.temporal.activity.ActivityInfo; -import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponseOrBuilder; - -public class ActivityPollResponseToInfo { - public static ActivityInfo toActivityInfoImpl( - PollActivityTaskQueueResponseOrBuilder response, - String namespace, - String activityTaskQueue, - boolean local) { - return new ActivityInfoImpl(response, namespace, activityTaskQueue, local, null); - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityPollTask.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityPollTask.java index 383b98629..a08dc38ca 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityPollTask.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityPollTask.java @@ -30,12 +30,11 @@ import io.temporal.api.workflowservice.v1.GetSystemInfoResponse; import io.temporal.api.workflowservice.v1.PollActivityTaskQueueRequest; import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse; -import io.temporal.internal.activity.ActivityPollResponseToInfo; import io.temporal.internal.common.ProtobufTimeUtils; import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.worker.MetricsType; -import io.temporal.worker.tuning.*; import java.util.Objects; +import java.util.concurrent.Semaphore; import java.util.function.Supplier; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -46,7 +45,7 @@ final class ActivityPollTask implements Poller.PollTask { private static final Logger log = LoggerFactory.getLogger(ActivityPollTask.class); private final WorkflowServiceStubs service; - private final TrackingSlotSupplier slotSupplier; + private final Semaphore pollSemaphore; private final Scope metricsScope; private final PollActivityTaskQueueRequest pollRequest; @@ -58,11 +57,11 @@ public ActivityPollTask( @Nullable String buildId, boolean useBuildIdForVersioning, double activitiesPerSecond, - @Nonnull TrackingSlotSupplier slotSupplier, + Semaphore pollSemaphore, @Nonnull Scope metricsScope, @Nonnull Supplier serverCapabilities) { this.service = Objects.requireNonNull(service); - this.slotSupplier = slotSupplier; + this.pollSemaphore = pollSemaphore; this.metricsScope = Objects.requireNonNull(metricsScope); PollActivityTaskQueueRequest.Builder pollRequest = @@ -93,22 +92,13 @@ public ActivityTask poll() { log.trace("poll request begin: " + pollRequest); } PollActivityTaskQueueResponse response; - SlotPermit permit; boolean isSuccessful = false; try { - permit = - slotSupplier.reserveSlot( - new SlotReservationData( - pollRequest.getTaskQueue().getName(), - pollRequest.getIdentity(), - pollRequest.getWorkerVersionCapabilities().getBuildId())); + pollSemaphore.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; - } catch (Exception e) { - log.warn("Error while trying to reserve a slot for an activity", e.getCause()); - return null; } try { @@ -128,20 +118,9 @@ public ActivityTask poll() { ProtobufTimeUtils.toM3Duration( response.getStartedTime(), response.getCurrentAttemptScheduledTime())); isSuccessful = true; - slotSupplier.markSlotUsed( - new ActivitySlotInfo( - ActivityPollResponseToInfo.toActivityInfoImpl( - response, - pollRequest.getNamespace(), - pollRequest.getTaskQueue().getNormalName(), - false), - pollRequest.getIdentity(), - pollRequest.getWorkerVersionCapabilities().getBuildId()), - permit); - return new ActivityTask( - response, () -> slotSupplier.releaseSlot(SlotReleaseReason.taskComplete(), permit)); + return new ActivityTask(response, pollSemaphore::release); } finally { - if (!isSuccessful) slotSupplier.releaseSlot(SlotReleaseReason.neverUsed(), permit); + if (!isSuccessful) pollSemaphore.release(); } } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityWorker.java index 70a54425f..73b037f21 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityWorker.java @@ -39,10 +39,9 @@ import io.temporal.serviceclient.rpcretry.DefaultStubServiceOperationRpcRetryOptions; import io.temporal.worker.MetricsType; import io.temporal.worker.WorkerMetricsTag; -import io.temporal.worker.tuning.*; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; import org.slf4j.Logger; @@ -65,7 +64,8 @@ final class ActivityWorker implements SuspendableWorker { private final Scope workerMetricsScope; private final GrpcRetryer grpcRetryer; private final GrpcRetryer.GrpcRetryerOptions replyGrpcRetryerOptions; - private final TrackingSlotSupplier slotSupplier; + private final int executorSlots; + private final Semaphore executorSlotsSemaphore; public ActivityWorker( @Nonnull WorkflowServiceStubs service, @@ -73,8 +73,7 @@ public ActivityWorker( @Nonnull String taskQueue, double taskQueueActivitiesPerSecond, @Nonnull SingleWorkerOptions options, - @Nonnull ActivityTaskHandler handler, - @Nonnull TrackingSlotSupplier slotSupplier) { + @Nonnull ActivityTaskHandler handler) { this.service = Objects.requireNonNull(service); this.namespace = Objects.requireNonNull(namespace); this.taskQueue = Objects.requireNonNull(taskQueue); @@ -88,8 +87,8 @@ public ActivityWorker( this.replyGrpcRetryerOptions = new GrpcRetryer.GrpcRetryerOptions( DefaultStubServiceOperationRpcRetryOptions.INSTANCE, null); - this.slotSupplier = slotSupplier; - this.slotSupplier.setMetricsScope(this.workerMetricsScope); + this.executorSlots = options.getTaskExecutorThreadPoolSize(); + this.executorSlotsSemaphore = new Semaphore(executorSlots); } @Override @@ -102,7 +101,8 @@ public boolean start() { options.getIdentity(), new TaskHandlerImpl(handler), pollerOptions, - slotSupplier.maximumSlots(), + options.getTaskExecutorThreadPoolSize(), + workerMetricsScope, true); poller = new Poller<>( @@ -115,7 +115,7 @@ public boolean start() { options.getBuildId(), options.isUsingBuildIdForVersioning(), taskQueueActivitiesPerSecond, - this.slotSupplier, + executorSlotsSemaphore, workerMetricsScope, service.getServerCapabilities()), this.pollTaskExecutor, @@ -131,14 +131,14 @@ public boolean start() { @Override public CompletableFuture shutdown(ShutdownManager shutdownManager, boolean interruptTasks) { - String supplierName = this + "#executorSlots"; + String semaphoreName = this + "#executorSlotsSemaphore"; return poller .shutdown(shutdownManager, interruptTasks) .thenCompose( ignore -> !interruptTasks - ? shutdownManager.waitForSupplierPermitsReleasedUnlimited( - slotSupplier, supplierName) + ? shutdownManager.waitForSemaphorePermitsReleaseUntimed( + executorSlotsSemaphore, executorSlots, semaphoreName) : CompletableFuture.completedFuture(null)) .thenCompose( ignore -> @@ -416,33 +416,23 @@ private void logExceptionDuringResultReporting( private final class EagerActivityDispatcherImpl implements EagerActivityDispatcher { @Override - public Optional tryReserveActivitySlot( + public boolean tryReserveActivitySlot( ScheduleActivityTaskCommandAttributesOrBuilder commandAttributes) { - if (!WorkerLifecycleState.ACTIVE.equals(ActivityWorker.this.getLifecycleState()) - || !Objects.equals( - commandAttributes.getTaskQueue().getName(), ActivityWorker.this.taskQueue)) { - return Optional.empty(); - } - return ActivityWorker.this.slotSupplier.tryReserveSlot( - new SlotReservationData( - ActivityWorker.this.taskQueue, options.getIdentity(), options.getBuildId())); + return WorkerLifecycleState.ACTIVE.equals(ActivityWorker.this.getLifecycleState()) + && Objects.equals( + commandAttributes.getTaskQueue().getName(), ActivityWorker.this.taskQueue) + && ActivityWorker.this.executorSlotsSemaphore.tryAcquire(); } @Override - public void releaseActivitySlotReservations(Iterable permits) { - for (SlotPermit permit : permits) { - ActivityWorker.this.slotSupplier.releaseSlot(SlotReleaseReason.neverUsed(), permit); - } + public void releaseActivitySlotReservations(int slotCounts) { + ActivityWorker.this.executorSlotsSemaphore.release(slotCounts); } @Override - public void dispatchActivity(PollActivityTaskQueueResponse activity, SlotPermit permit) { + public void dispatchActivity(PollActivityTaskQueueResponse activity) { ActivityWorker.this.pollTaskExecutor.process( - new ActivityTask( - activity, - () -> - ActivityWorker.this.slotSupplier.releaseSlot( - SlotReleaseReason.taskComplete(), permit))); + new ActivityTask(activity, ActivityWorker.this.executorSlotsSemaphore::release)); } } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivityDispatcher.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivityDispatcher.java index 92b0b6e88..721bdfa79 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivityDispatcher.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivityDispatcher.java @@ -22,33 +22,30 @@ import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributesOrBuilder; import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse; -import io.temporal.worker.tuning.SlotPermit; -import java.util.Optional; public interface EagerActivityDispatcher { - Optional tryReserveActivitySlot( - ScheduleActivityTaskCommandAttributesOrBuilder commandAttributes); + boolean tryReserveActivitySlot(ScheduleActivityTaskCommandAttributesOrBuilder commandAttributes); - void releaseActivitySlotReservations(Iterable permits); + void releaseActivitySlotReservations(int slotCounts); - void dispatchActivity(PollActivityTaskQueueResponse activity, SlotPermit permit); + void dispatchActivity(PollActivityTaskQueueResponse activity); class NoopEagerActivityDispatcher implements EagerActivityDispatcher { @Override - public Optional tryReserveActivitySlot( + public boolean tryReserveActivitySlot( ScheduleActivityTaskCommandAttributesOrBuilder commandAttributes) { - return Optional.empty(); + return false; } @Override - public void releaseActivitySlotReservations(Iterable permits) { - if (permits.iterator().hasNext()) + public void releaseActivitySlotReservations(int slotCounts) { + if (slotCounts > 0) throw new IllegalStateException( "Trying to release activity slots on a NoopEagerActivityDispatcher"); } @Override - public void dispatchActivity(PollActivityTaskQueueResponse activity, SlotPermit permit) { + public void dispatchActivity(PollActivityTaskQueueResponse activity) { throw new IllegalStateException( "Trying to dispatch activity on a NoopEagerActivityDispatcher"); } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivitySlotsReservation.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivitySlotsReservation.java index e1a5c1e88..2da101321 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivitySlotsReservation.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/EagerActivitySlotsReservation.java @@ -28,18 +28,14 @@ import io.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedRequest; import io.temporal.api.workflowservice.v1.RespondWorkflowTaskCompletedResponse; import io.temporal.internal.Config; -import io.temporal.worker.tuning.SlotPermit; import java.io.Closeable; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; import javax.annotation.concurrent.NotThreadSafe; /** This class is not thread safe and shouldn't leave the boundaries of one activity executor */ @NotThreadSafe class EagerActivitySlotsReservation implements Closeable { private final EagerActivityDispatcher eagerActivityDispatcher; - private final List reservedSlots = new ArrayList<>(); + private int outstandingReservationSlotsCount = 0; EagerActivitySlotsReservation(EagerActivityDispatcher eagerActivityDispatcher) { this.eagerActivityDispatcher = eagerActivityDispatcher; @@ -53,14 +49,10 @@ public void applyToRequest(RespondWorkflowTaskCompletedRequest.Builder mutableRe ScheduleActivityTaskCommandAttributes commandAttributes = command.getScheduleActivityTaskCommandAttributes(); if (!commandAttributes.getRequestEagerExecution()) continue; - boolean atLimit = this.reservedSlots.size() >= Config.EAGER_ACTIVITIES_LIMIT; - Optional permit = Optional.empty(); - if (!atLimit) { - permit = this.eagerActivityDispatcher.tryReserveActivitySlot(commandAttributes); - } - if (permit.isPresent()) { - this.reservedSlots.add(permit.get()); + if (this.outstandingReservationSlotsCount < Config.EAGER_ACTIVITIES_LIMIT + && this.eagerActivityDispatcher.tryReserveActivitySlot(commandAttributes)) { + this.outstandingReservationSlotsCount++; } else { mutableRequest.setCommands( i, @@ -74,32 +66,34 @@ public void applyToRequest(RespondWorkflowTaskCompletedRequest.Builder mutableRe public void handleResponse(RespondWorkflowTaskCompletedResponse serverResponse) { int activityTasksCount = serverResponse.getActivityTasksCount(); Preconditions.checkArgument( - activityTasksCount <= this.reservedSlots.size(), + activityTasksCount <= this.outstandingReservationSlotsCount, "Unexpectedly received %s eager activities though we only requested %s", activityTasksCount, - this.reservedSlots.size()); + this.outstandingReservationSlotsCount); + + releaseSlots(this.outstandingReservationSlotsCount - activityTasksCount); for (PollActivityTaskQueueResponse act : serverResponse.getActivityTasksList()) { - // don't release slots here, instead the release function is called in the activity worker to - // release when the activity is done - SlotPermit permit = this.reservedSlots.remove(0); - this.eagerActivityDispatcher.dispatchActivity(act, permit); + // don't release slots here, instead the semaphore release reference is passed to the activity + // worker to release when the activity is done + this.eagerActivityDispatcher.dispatchActivity(act); } - // Release any remaining that we won't be using - try { - this.eagerActivityDispatcher.releaseActivitySlotReservations(this.reservedSlots); - } finally { - this.reservedSlots.clear(); - } + this.outstandingReservationSlotsCount = 0; } @Override public void close() { - if (!this.reservedSlots.isEmpty()) { - // Release all slots - this.eagerActivityDispatcher.releaseActivitySlotReservations(this.reservedSlots); - this.reservedSlots.clear(); - } + if (this.outstandingReservationSlotsCount > 0) + releaseSlots(this.outstandingReservationSlotsCount); + } + + private void releaseSlots(int slotsToRelease) { + if (slotsToRelease > this.outstandingReservationSlotsCount) + throw new IllegalStateException( + "Trying to release more activity slots than outstanding reservations"); + + this.eagerActivityDispatcher.releaseActivitySlotReservations(slotsToRelease); + this.outstandingReservationSlotsCount -= slotsToRelease; } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityAttemptTask.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityAttemptTask.java index ec38c5b6e..146f56333 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityAttemptTask.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityAttemptTask.java @@ -21,6 +21,7 @@ package io.temporal.internal.worker; import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse; +import io.temporal.workflow.Functions; import java.util.concurrent.ScheduledFuture; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -28,14 +29,17 @@ class LocalActivityAttemptTask { private final @Nonnull LocalActivityExecutionContext executionContext; private final @Nonnull PollActivityTaskQueueResponse.Builder attemptTask; + private final @Nullable Functions.Proc takenFromQueueCallback; private final @Nullable ScheduledFuture scheduleToStartFuture; public LocalActivityAttemptTask( @Nonnull LocalActivityExecutionContext executionContext, @Nonnull PollActivityTaskQueueResponse.Builder attemptTask, + @Nullable Functions.Proc takenFromQueueCallback, @Nullable ScheduledFuture scheduleToStartFuture) { this.executionContext = executionContext; this.attemptTask = attemptTask; + this.takenFromQueueCallback = takenFromQueueCallback; this.scheduleToStartFuture = scheduleToStartFuture; } @@ -53,6 +57,13 @@ public PollActivityTaskQueueResponse.Builder getAttemptTask() { return attemptTask; } + public void markAsTakenFromQueue() { + executionContext.newAttempt(); + if (takenFromQueueCallback != null) { + takenFromQueueCallback.apply(); + } + } + @Nullable public ScheduledFuture getScheduleToStartFuture() { return scheduleToStartFuture; diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityExecutionContext.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityExecutionContext.java index 356b7ad45..061415b01 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityExecutionContext.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityExecutionContext.java @@ -26,9 +26,6 @@ import io.temporal.api.workflowservice.v1.DescribeWorkflowExecutionRequest; import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse; import io.temporal.internal.statemachines.ExecuteLocalActivityParameters; -import io.temporal.worker.tuning.LocalActivitySlotInfo; -import io.temporal.worker.tuning.SlotPermit; -import io.temporal.worker.tuning.SlotReleaseReason; import io.temporal.workflow.Functions; import java.time.Duration; import java.util.Objects; @@ -47,20 +44,16 @@ class LocalActivityExecutionContext { private @Nullable ScheduledFuture scheduleToCloseFuture; private final @Nonnull CompletableFuture executionResult = new CompletableFuture<>(); - private @Nullable SlotPermit permit; - private final TrackingSlotSupplier slotSupplier; public LocalActivityExecutionContext( @Nonnull ExecuteLocalActivityParameters executionParams, @Nonnull Functions.Proc1 resultCallback, - @Nullable Deadline scheduleToCloseDeadline, - TrackingSlotSupplier slotSupplier) { + @Nullable Deadline scheduleToCloseDeadline) { this.executionParams = Objects.requireNonNull(executionParams, "executionParams"); this.executionResult.thenAccept( Objects.requireNonNull(resultCallback, "resultCallback")::apply); this.scheduleToCloseDeadline = scheduleToCloseDeadline; this.currentAttempt = new AtomicInteger(executionParams.getInitialAttempt()); - this.slotSupplier = slotSupplier; Failure previousExecutionFailure = executionParams.getPreviousLocalExecutionFailure(); if (previousExecutionFailure != null) { if (previousExecutionFailure.hasTimeoutFailureInfo() && previousExecutionFailure.hasCause()) { @@ -161,11 +154,6 @@ public boolean callback(LocalActivityResult result) { if (scheduleToCloseFuture != null) { scheduleToCloseFuture.cancel(false); } - SlotReleaseReason reason = SlotReleaseReason.taskComplete(); - if (result.getProcessingError() != null) { - reason = SlotReleaseReason.error(new Exception(result.getProcessingError().getThrowable())); - } - slotSupplier.releaseSlot(reason, permit); return executionResult.complete(result); } @@ -176,13 +164,4 @@ public boolean isCompleted() { public void newAttempt() { executionParams.getOnNewAttemptCallback().apply(); } - - public void setPermit(SlotPermit permit) { - this.permit = permit; - } - - @Nullable - public SlotPermit getPermit() { - return permit; - } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityWorker.java index da55f4e03..5c6254a7c 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/LocalActivityWorker.java @@ -24,8 +24,6 @@ import static io.temporal.internal.worker.LocalActivityResult.processingFailed; import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.SimpleTimeLimiter; -import com.google.common.util.concurrent.TimeLimiter; import com.uber.m3.tally.Scope; import com.uber.m3.tally.Stopwatch; import com.uber.m3.util.ImmutableMap; @@ -38,7 +36,6 @@ import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponseOrBuilder; import io.temporal.common.RetryOptions; import io.temporal.failure.ApplicationFailure; -import io.temporal.internal.activity.ActivityPollResponseToInfo; import io.temporal.internal.common.ProtobufTimeUtils; import io.temporal.internal.common.RetryOptionsUtils; import io.temporal.internal.logging.LoggerTag; @@ -46,7 +43,6 @@ import io.temporal.serviceclient.MetricsTag; import io.temporal.worker.MetricsType; import io.temporal.worker.WorkerMetricsTag; -import io.temporal.worker.tuning.*; import io.temporal.workflow.Functions; import java.time.Duration; import java.util.Objects; @@ -74,36 +70,33 @@ final class LocalActivityWorker implements Startable, Shutdownable { private ScheduledExecutorService scheduledExecutor; private PollTaskExecutor activityAttemptTaskExecutor; - private final TrackingSlotSupplier slotSupplier; public LocalActivityWorker( @Nonnull String namespace, @Nonnull String taskQueue, @Nonnull SingleWorkerOptions options, - @Nonnull ActivityTaskHandler handler, - @Nonnull TrackingSlotSupplier slotSupplier) { + @Nonnull ActivityTaskHandler handler) { this.namespace = Objects.requireNonNull(namespace); this.taskQueue = Objects.requireNonNull(taskQueue); this.handler = handler; - this.slotSupplier = Objects.requireNonNull(slotSupplier); - this.laScheduler = new LocalActivityDispatcherImpl(slotSupplier); + this.laScheduler = new LocalActivityDispatcherImpl(2 * options.getTaskExecutorThreadPoolSize()); this.options = Objects.requireNonNull(options); this.pollerOptions = getPollerOptions(options); this.workerMetricsScope = MetricsTag.tagged( options.getMetricsScope(), WorkerMetricsTag.WorkerType.LOCAL_ACTIVITY_WORKER); - this.slotSupplier.setMetricsScope(this.workerMetricsScope); } private void submitRetry( @Nonnull LocalActivityExecutionContext executionContext, @Nonnull PollActivityTaskQueueResponse.Builder activityTask) { - submitAttempt(executionContext, activityTask); + submitAttempt(executionContext, activityTask, null); } private void submitAttempt( @Nonnull LocalActivityExecutionContext executionContext, - @Nonnull PollActivityTaskQueueResponse.Builder activityTask) { + @Nonnull PollActivityTaskQueueResponse.Builder activityTask, + @Nullable Functions.Proc leftQueueCallback) { @Nullable Duration scheduleToStartTimeout = executionContext.getScheduleToStartTimeout(); @Nullable ScheduledFuture scheduleToStartFuture = null; if (scheduleToStartTimeout != null) { @@ -116,7 +109,8 @@ private void submitAttempt( activityTask.setCurrentAttemptScheduledTime(ProtobufTimeUtils.getCurrentProtoTime()); LocalActivityAttemptTask task = - new LocalActivityAttemptTask(executionContext, activityTask, scheduleToStartFuture); + new LocalActivityAttemptTask( + executionContext, activityTask, leftQueueCallback, scheduleToStartFuture); activityAttemptTaskExecutor.process(task); } @@ -201,12 +195,13 @@ private class LocalActivityDispatcherImpl implements LocalActivityDispatcher { * Retries always get a green light, but we have a backpressure for new tasks if the queue fills * up with not picked up new executions */ - private final TrackingSlotSupplier slotSupplier; + private final Semaphore newExecutionsBackpressureSemaphore; - public LocalActivityDispatcherImpl(TrackingSlotSupplier slotSupplier) { - // we allow submitters to block and wait till the workflow task heartbeat to allow the worker - // to tolerate spikes of short local activity executions. - this.slotSupplier = slotSupplier; + public LocalActivityDispatcherImpl(int semaphorePermits) { + // number of permits for this semaphore is not that important, because we allow submitter to + // block and wait till the workflow task heartbeat to allow the worker to tolerate spikes of + // short local activity executions. + this.newExecutionsBackpressureSemaphore = new Semaphore(semaphorePermits); } @Override @@ -245,8 +240,7 @@ public boolean dispatch( } LocalActivityExecutionContext executionContext = - new LocalActivityExecutionContext( - params, resultCallback, scheduleToCloseDeadline, slotSupplier); + new LocalActivityExecutionContext(params, resultCallback, scheduleToCloseDeadline); PollActivityTaskQueueResponse.Builder activityTask = executionContext.getInitialTask(); @@ -262,69 +256,51 @@ public boolean dispatch( private boolean submitANewExecution( @Nonnull LocalActivityExecutionContext executionContext, @Nonnull PollActivityTaskQueueResponse.Builder activityTask, - @Nullable Deadline heartbeatDeadline) { - long acceptanceTimeoutMs = 0; - boolean timeoutIsScheduleToStart = false; - if (heartbeatDeadline != null) { - acceptanceTimeoutMs = heartbeatDeadline.timeRemaining(TimeUnit.MILLISECONDS); - } - Duration scheduleToStartTimeout = executionContext.getScheduleToStartTimeout(); - if (scheduleToStartTimeout != null) { - long scheduleToStartTimeoutMs = scheduleToStartTimeout.toMillis(); - if (scheduleToStartTimeoutMs > 0 && scheduleToStartTimeoutMs < acceptanceTimeoutMs) { - acceptanceTimeoutMs = scheduleToStartTimeoutMs; - timeoutIsScheduleToStart = true; - } - } + @Nullable Deadline acceptanceDeadline) { try { - SlotPermit permit = null; - SlotReservationData reservationCtx = - new SlotReservationData(taskQueue, options.getIdentity(), options.getBuildId()); - if (acceptanceTimeoutMs <= 0) { - permit = slotSupplier.reserveSlot(reservationCtx); + boolean accepted; + if (acceptanceDeadline == null) { + newExecutionsBackpressureSemaphore.acquire(); + accepted = true; } else { - try { - TimeLimiter timeLimiter = SimpleTimeLimiter.create(Executors.newCachedThreadPool()); - timeLimiter.callWithTimeout( - () -> slotSupplier.reserveSlot(reservationCtx), - acceptanceTimeoutMs, - TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - // In the event that we timed out waiting for a permit *because of schedule to start* we - // still want to proceed with the "attempt" with a null permit, which will then - // immediately fail with the s2s timeout. - if (!timeoutIsScheduleToStart) { - log.warn( - "LocalActivity queue is full and submitting timed out for activity {} with acceptanceTimeoutMs: {}", - activityTask.getActivityId(), - acceptanceTimeoutMs); - return false; - } + long acceptanceTimeoutMs = acceptanceDeadline.timeRemaining(TimeUnit.MILLISECONDS); + if (acceptanceTimeoutMs > 0) { + accepted = + newExecutionsBackpressureSemaphore.tryAcquire( + acceptanceTimeoutMs, TimeUnit.MILLISECONDS); + } else { + accepted = newExecutionsBackpressureSemaphore.tryAcquire(); + } + if (!accepted) { + log.warn( + "LocalActivity queue is full and submitting timed out for activity {} with acceptanceTimeoutMs: {}", + activityTask.getActivityId(), + acceptanceTimeoutMs); } } - executionContext.setPermit(permit); - // we should publish scheduleToClose before submission, so the handlers always see a full - // state of executionContext - @Nullable Deadline scheduleToCloseDeadline = executionContext.getScheduleToCloseDeadline(); - if (scheduleToCloseDeadline != null) { - ScheduledFuture scheduleToCloseFuture = - scheduledExecutor.schedule( - new FinalTimeoutHandler( - TimeoutType.TIMEOUT_TYPE_SCHEDULE_TO_CLOSE, executionContext), - scheduleToCloseDeadline.timeRemaining(TimeUnit.MILLISECONDS), - TimeUnit.MILLISECONDS); - executionContext.setScheduleToCloseFuture(scheduleToCloseFuture); + if (accepted) { + // we should publish scheduleToClose before submission, so the handlers always see a full + // state of executionContext + @Nullable + Deadline scheduleToCloseDeadline = executionContext.getScheduleToCloseDeadline(); + if (scheduleToCloseDeadline != null) { + ScheduledFuture scheduleToCloseFuture = + scheduledExecutor.schedule( + new FinalTimeoutHandler( + TimeoutType.TIMEOUT_TYPE_SCHEDULE_TO_CLOSE, executionContext), + scheduleToCloseDeadline.timeRemaining(TimeUnit.MILLISECONDS), + TimeUnit.MILLISECONDS); + executionContext.setScheduleToCloseFuture(scheduleToCloseFuture); + } + submitAttempt( + executionContext, activityTask, newExecutionsBackpressureSemaphore::release); + log.trace("LocalActivity queued: {}", activityTask.getActivityId()); } - submitAttempt(executionContext, activityTask); - log.trace("LocalActivity queued: {}", activityTask.getActivityId()); - return true; + return accepted; } catch (InterruptedException e) { Thread.currentThread().interrupt(); return false; - } catch (Exception e) { - log.warn("Error while trying to reserve a slot for local activity", e.getCause()); - return false; } } @@ -440,13 +416,14 @@ private AttemptTaskHandlerImpl(ActivityTaskHandler handler) { @Override public void handle(LocalActivityAttemptTask attemptTask) throws Exception { + attemptTask.markAsTakenFromQueue(); + // cancel scheduleToStart timeout if not already fired @Nullable ScheduledFuture scheduleToStartFuture = attemptTask.getScheduleToStartFuture(); boolean scheduleToStartFired = scheduleToStartFuture != null && !scheduleToStartFuture.cancel(false); LocalActivityExecutionContext executionContext = attemptTask.getExecutionContext(); - executionContext.newAttempt(); PollActivityTaskQueueResponseOrBuilder activityTask = attemptTask.getAttemptTask(); // if an activity was already completed by any mean like scheduleToClose or scheduleToStart, @@ -472,15 +449,6 @@ public void handle(LocalActivityAttemptTask attemptTask) throws Exception { MDC.put(LoggerTag.WORKFLOW_ID, activityTask.getWorkflowExecution().getWorkflowId()); MDC.put(LoggerTag.WORKFLOW_TYPE, activityTask.getWorkflowType().getName()); MDC.put(LoggerTag.RUN_ID, activityTask.getWorkflowExecution().getRunId()); - - slotSupplier.markSlotUsed( - new LocalActivitySlotInfo( - ActivityPollResponseToInfo.toActivityInfoImpl( - activityTask, namespace, taskQueue, true), - options.getIdentity(), - options.getBuildId()), - executionContext.getPermit()); - try { ScheduledFuture startToCloseTimeoutFuture = null; @@ -709,7 +677,8 @@ public boolean start() { options.getIdentity(), new AttemptTaskHandlerImpl(handler), pollerOptions, - slotSupplier.maximumSlots(), + options.getTaskExecutorThreadPoolSize(), + workerMetricsScope, false); this.workerMetricsScope.counter(MetricsType.WORKER_START_COUNTER).inc(1); diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/PollTaskExecutor.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/PollTaskExecutor.java index 38cdfcd21..08c65f92f 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/PollTaskExecutor.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/PollTaskExecutor.java @@ -21,9 +21,12 @@ package io.temporal.internal.worker; import com.google.common.base.Preconditions; +import com.uber.m3.tally.Scope; import io.temporal.internal.logging.LoggerTag; +import io.temporal.worker.MetricsType; import java.util.Objects; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.slf4j.MDC; @@ -40,8 +43,10 @@ public interface TaskHandler { private final String identity; private final TaskHandler handler; private final PollerOptions pollerOptions; + private final Scope metricsScope; private final ThreadPoolExecutor taskExecutor; + private final AtomicInteger availableTaskSlots; private final String pollThreadNamePrefix; PollTaskExecutor( @@ -51,12 +56,14 @@ public interface TaskHandler { @Nonnull TaskHandler handler, @Nonnull PollerOptions pollerOptions, int workerTaskSlots, + @Nonnull Scope metricsScope, boolean synchronousQueue) { this.namespace = Objects.requireNonNull(namespace); this.taskQueue = Objects.requireNonNull(taskQueue); this.identity = Objects.requireNonNull(identity); this.handler = Objects.requireNonNull(handler); this.pollerOptions = Objects.requireNonNull(pollerOptions); + this.metricsScope = Objects.requireNonNull(metricsScope); this.taskExecutor = new ThreadPoolExecutor( @@ -72,6 +79,9 @@ public interface TaskHandler { synchronousQueue ? new SynchronousQueue<>() : new LinkedBlockingQueue<>()); this.taskExecutor.allowCoreThreadTimeOut(true); + this.availableTaskSlots = new AtomicInteger(workerTaskSlots); + publishSlotsMetric(); + this.pollThreadNamePrefix = pollerOptions.getPollThreadNamePrefix().replaceFirst("Poller", "Executor"); @@ -87,6 +97,8 @@ public void process(@Nonnull T task) { Preconditions.checkNotNull(task, "task"); taskExecutor.execute( () -> { + availableTaskSlots.decrementAndGet(); + publishSlotsMetric(); try { MDC.put(LoggerTag.NAMESPACE, namespace); MDC.put(LoggerTag.TASK_QUEUE, taskQueue); @@ -105,6 +117,8 @@ public void process(@Nonnull T task) { // throw (Error)e; // } } finally { + availableTaskSlots.incrementAndGet(); + publishSlotsMetric(); MDC.remove(LoggerTag.NAMESPACE); MDC.remove(LoggerTag.TASK_QUEUE); } @@ -142,4 +156,10 @@ public String toString() { // that will simplify such kind of logging through workers. return String.format("PollTaskExecutor{name=%s, identity=%s}", pollThreadNamePrefix, identity); } + + private void publishSlotsMetric() { + this.metricsScope + .gauge(MetricsType.WORKER_TASK_SLOTS_AVAILABLE) + .update(availableTaskSlots.get()); + } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/ShutdownManager.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/ShutdownManager.java index f790febb4..61c5c85a2 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/ShutdownManager.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/ShutdownManager.java @@ -68,10 +68,12 @@ public CompletableFuture shutdownExecutorUntimed( return untimedWait(executorToShutdown, executorName); } - public CompletableFuture waitForSupplierPermitsReleasedUnlimited( - TrackingSlotSupplier slotSupplier, String name) { + public CompletableFuture waitForSemaphorePermitsReleaseUntimed( + Semaphore semaphore, int initialSemaphorePermits, String semaphoreName) { CompletableFuture future = new CompletableFuture<>(); - scheduledExecutorService.submit(new SlotSupplierDelayShutdown(slotSupplier, name, future)); + scheduledExecutorService.submit( + new SemaphoreReportingDelayShutdown( + semaphore, initialSemaphorePermits, semaphoreName, future)); return future; } @@ -265,25 +267,30 @@ void onSlowSuccessfulTermination() { } } - private class SlotSupplierDelayShutdown extends ReportingDelayShutdown { - private final TrackingSlotSupplier slotSupplier; - private final String name; + private class SemaphoreReportingDelayShutdown extends ReportingDelayShutdown { + private final Semaphore semaphore; + private final int initialSemaphorePermits; + private final String semaphoreName; - public SlotSupplierDelayShutdown( - TrackingSlotSupplier supplier, String name, CompletableFuture promise) { + public SemaphoreReportingDelayShutdown( + Semaphore semaphore, + int initialSemaphorePermits, + String semaphoreName, + CompletableFuture promise) { super(promise); - this.slotSupplier = supplier; - this.name = name; + this.semaphore = semaphore; + this.initialSemaphorePermits = initialSemaphorePermits; + this.semaphoreName = semaphoreName; } @Override boolean isTerminated() { - return slotSupplier.getIssuedSlots() == 0; + return semaphore.availablePermits() == initialSemaphorePermits; } @Override void onSlowTermination() { - log.warn("Wait for release of slots of {} takes a long time", name); + log.warn("Wait for release of slots of {} takes a long time", semaphoreName); } @Override @@ -291,7 +298,7 @@ void onSuccessfulTermination() {} @Override void onSlowSuccessfulTermination() { - log.warn("All slots of {} were successfully released", name); + log.warn("All slots of {} were successfully released", semaphoreName); } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SingleWorkerOptions.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SingleWorkerOptions.java index 990a9ce46..3a6f998dd 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SingleWorkerOptions.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SingleWorkerOptions.java @@ -47,11 +47,13 @@ public static final class Builder { private String buildId; private boolean useBuildIdForVersioning; private DataConverter dataConverter; + private int taskExecutorThreadPoolSize = 100; private PollerOptions pollerOptions; private Scope metricsScope; private boolean enableLoggingInReplay; private List contextPropagators; private WorkerInterceptor[] workerInterceptors; + private Duration stickyQueueScheduleToStartTimeout; private long defaultDeadlockDetectionTimeout; private Duration maxHeartbeatThrottleInterval; @@ -68,6 +70,7 @@ private Builder(SingleWorkerOptions options) { this.binaryChecksum = options.getBinaryChecksum(); this.dataConverter = options.getDataConverter(); this.pollerOptions = options.getPollerOptions(); + this.taskExecutorThreadPoolSize = options.getTaskExecutorThreadPoolSize(); this.metricsScope = options.getMetricsScope(); this.enableLoggingInReplay = options.getEnableLoggingInReplay(); this.contextPropagators = options.getContextPropagators(); @@ -97,6 +100,11 @@ public Builder setDataConverter(DataConverter dataConverter) { return this; } + public Builder setTaskExecutorThreadPoolSize(int taskExecutorThreadPoolSize) { + this.taskExecutorThreadPoolSize = taskExecutorThreadPoolSize; + return this; + } + public Builder setPollerOptions(PollerOptions pollerOptions) { this.pollerOptions = pollerOptions; return this; @@ -187,6 +195,7 @@ public SingleWorkerOptions build() { this.buildId, this.useBuildIdForVersioning, dataConverter, + this.taskExecutorThreadPoolSize, pollerOptions, metricsScope, this.enableLoggingInReplay, @@ -205,6 +214,7 @@ public SingleWorkerOptions build() { private final String buildId; private final boolean useBuildIdForVersioning; private final DataConverter dataConverter; + private final int taskExecutorThreadPoolSize; private final PollerOptions pollerOptions; private final Scope metricsScope; private final boolean enableLoggingInReplay; @@ -222,6 +232,7 @@ private SingleWorkerOptions( String buildId, boolean useBuildIdForVersioning, DataConverter dataConverter, + int taskExecutorThreadPoolSize, PollerOptions pollerOptions, Scope metricsScope, boolean enableLoggingInReplay, @@ -237,6 +248,7 @@ private SingleWorkerOptions( this.buildId = buildId; this.useBuildIdForVersioning = useBuildIdForVersioning; this.dataConverter = dataConverter; + this.taskExecutorThreadPoolSize = taskExecutorThreadPoolSize; this.pollerOptions = pollerOptions; this.metricsScope = metricsScope; this.enableLoggingInReplay = enableLoggingInReplay; @@ -277,6 +289,10 @@ public DataConverter getDataConverter() { return dataConverter; } + public int getTaskExecutorThreadPoolSize() { + return taskExecutorThreadPoolSize; + } + public PollerOptions getPollerOptions() { return pollerOptions; } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SlotReservationData.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SlotReservationData.java deleted file mode 100644 index e3378d5c0..000000000 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SlotReservationData.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.internal.worker; - -public class SlotReservationData { - public final String taskQueue; - public final String workerIdentity; - public final String workerBuildId; - - public SlotReservationData(String taskQueue, String workerIdentity, String workerBuildId) { - this.taskQueue = taskQueue; - this.workerIdentity = workerIdentity; - this.workerBuildId = workerBuildId; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java index d41924473..8e1537b7f 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java @@ -24,7 +24,6 @@ import io.temporal.internal.activity.ActivityExecutionContextFactoryImpl; import io.temporal.internal.activity.ActivityTaskHandlerImpl; import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.worker.tuning.ActivitySlotInfo; import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; @@ -50,8 +49,7 @@ public SyncActivityWorker( String namespace, String taskQueue, double taskQueueActivitiesPerSecond, - SingleWorkerOptions options, - TrackingSlotSupplier slotSupplier) { + SingleWorkerOptions options) { this.identity = options.getIdentity(); this.namespace = namespace; this.taskQueue = taskQueue; @@ -85,13 +83,7 @@ public SyncActivityWorker( options.getContextPropagators()); this.worker = new ActivityWorker( - service, - namespace, - taskQueue, - taskQueueActivitiesPerSecond, - options, - taskHandler, - slotSupplier); + service, namespace, taskQueue, taskQueueActivitiesPerSecond, options, taskHandler); } public void registerActivityImplementations(Object... activitiesImplementation) { diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java index edfadfab1..a1ece312f 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java @@ -34,8 +34,6 @@ import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.worker.WorkflowImplementationOptions; import io.temporal.worker.WorkflowTaskDispatchHandle; -import io.temporal.worker.tuning.LocalActivitySlotInfo; -import io.temporal.worker.tuning.WorkflowSlotInfo; import io.temporal.workflow.Functions.Func; import java.lang.reflect.Type; import java.time.Duration; @@ -82,9 +80,7 @@ public SyncWorkflowWorker( @Nonnull WorkflowExecutorCache cache, String stickyTaskQueueName, @Nonnull WorkflowThreadExecutor workflowThreadExecutor, - @Nonnull EagerActivityDispatcher eagerActivityDispatcher, - @Nonnull TrackingSlotSupplier slotSupplier, - @Nonnull TrackingSlotSupplier laSlotSupplier) { + @Nonnull EagerActivityDispatcher eagerActivityDispatcher) { this.identity = singleWorkerOptions.getIdentity(); this.namespace = namespace; this.taskQueue = taskQueue; @@ -108,9 +104,7 @@ public SyncWorkflowWorker( laActivityExecutionContextFactory, localActivityOptions.getWorkerInterceptors(), localActivityOptions.getContextPropagators()); - laWorker = - new LocalActivityWorker( - namespace, taskQueue, localActivityOptions, laTaskHandler, laSlotSupplier); + laWorker = new LocalActivityWorker(namespace, taskQueue, localActivityOptions, laTaskHandler); TaskQueue stickyTaskQueue = null; if (stickyTaskQueueName != null) { stickyTaskQueue = createStickyTaskQueue(stickyTaskQueueName, taskQueue); @@ -137,8 +131,7 @@ public SyncWorkflowWorker( runLocks, cache, taskHandler, - eagerActivityDispatcher, - slotSupplier); + eagerActivityDispatcher); // Exists to support Worker#replayWorkflowExecution functionality. // This handler has to be non-sticky to avoid evicting actual executions from the cache diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/TrackingSlotSupplier.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/TrackingSlotSupplier.java deleted file mode 100644 index 89ee9c6f8..000000000 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/TrackingSlotSupplier.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.internal.worker; - -import com.uber.m3.tally.NoopScope; -import com.uber.m3.tally.Scope; -import io.temporal.worker.MetricsType; -import io.temporal.worker.tuning.*; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Wraps a slot supplier and supplements it with additional tracking information that is useful to - * provide to all implementations. This type is used internally rather than {@link SlotSupplier} - * directly. - * - * @param The slot info type - */ -public class TrackingSlotSupplier { - private final SlotSupplier inner; - private final AtomicInteger issuedSlots = new AtomicInteger(); - private final Map usedSlots = new ConcurrentHashMap<>(); - private Scope metricsScope; - - public TrackingSlotSupplier(SlotSupplier inner) { - this.inner = inner; - metricsScope = new NoopScope(); - } - - public SlotPermit reserveSlot(SlotReservationData dat) throws InterruptedException { - SlotPermit p = inner.reserveSlot(createCtx(dat)); - issuedSlots.incrementAndGet(); - return p; - } - - public Optional tryReserveSlot(SlotReservationData dat) { - Optional p = inner.tryReserveSlot(createCtx(dat)); - if (p.isPresent()) { - issuedSlots.incrementAndGet(); - } - return p; - } - - public void markSlotUsed(SI slotInfo, SlotPermit permit) { - if (permit == null) { - return; - } - inner.markSlotUsed(new SlotMarkUsedContextImpl(slotInfo, permit)); - usedSlots.put(permit, slotInfo); - publishSlotsMetric(); - } - - public void releaseSlot(SlotReleaseReason reason, SlotPermit permit) { - if (permit == null) { - return; - } - SI slotInfo = usedSlots.get(permit); - inner.releaseSlot(new SlotReleaseContextImpl(reason, permit, slotInfo)); - issuedSlots.decrementAndGet(); - usedSlots.remove(permit); - publishSlotsMetric(); - } - - public int maximumSlots() { - return inner.getMaximumSlots(); - } - - public int getIssuedSlots() { - return issuedSlots.get(); - } - - public void setMetricsScope(Scope metricsScope) { - this.metricsScope = metricsScope; - } - - /** - * If any slot supplier is resource-based, we want to attach a metrics scope to the controller - * (before it's labelled with the worker type). - */ - public void attachMetricsToResourceController(Scope metricsScope) { - if (inner instanceof ResourceBasedSlotSupplier) { - ((ResourceBasedSlotSupplier) inner).getResourceController().setMetricsScope(metricsScope); - } - } - - Map getUsedSlots() { - return usedSlots; - } - - private void publishSlotsMetric() { - this.metricsScope - .gauge(MetricsType.WORKER_TASK_SLOTS_AVAILABLE) - .update(maximumSlots() - usedSlots.size()); - } - - private SlotReserveContext createCtx(SlotReservationData dat) { - return new SlotReserveContextImpl( - dat.taskQueue, - Collections.unmodifiableMap(usedSlots), - dat.workerIdentity, - dat.workerBuildId, - issuedSlots); - } - - private class SlotReserveContextImpl implements SlotReserveContext { - private final String taskQueue; - private final Map usedSlots; - private final String workerIdentity; - private final String workerBuildId; - private final AtomicInteger issuedSlots; - - private SlotReserveContextImpl( - String taskQueue, - Map usedSlots, - String workerIdentity, - String workerBuildId, - AtomicInteger issuedSlots) { - this.taskQueue = taskQueue; - this.usedSlots = usedSlots; - this.workerIdentity = workerIdentity; - this.workerBuildId = workerBuildId; - this.issuedSlots = issuedSlots; - } - - @Override - public String getTaskQueue() { - return taskQueue; - } - - @Override - public Map getUsedSlots() { - return usedSlots; - } - - @Override - public String getWorkerIdentity() { - return workerIdentity; - } - - @Override - public String getWorkerBuildId() { - return workerBuildId; - } - - @Override - public int getNumIssuedSlots() { - return issuedSlots.get(); - } - } - - private class SlotMarkUsedContextImpl implements SlotMarkUsedContext { - private final SI slotInfo; - private final SlotPermit slotPermit; - - protected SlotMarkUsedContextImpl(SI slotInfo, SlotPermit slotPermit) { - this.slotInfo = slotInfo; - this.slotPermit = slotPermit; - } - - @Override - public SI getSlotInfo() { - return slotInfo; - } - - @Override - public SlotPermit getSlotPermit() { - return slotPermit; - } - } - - private class SlotReleaseContextImpl implements SlotReleaseContext { - private final SlotPermit slotPermit; - private final SlotReleaseReason reason; - private final SI slotInfo; - - protected SlotReleaseContextImpl(SlotReleaseReason reason, SlotPermit slotPermit, SI slotInfo) { - this.slotPermit = slotPermit; - this.reason = reason; - this.slotInfo = slotInfo; - } - - @Override - public SlotReleaseReason getSlotReleaseReason() { - return reason; - } - - @Override - public SlotPermit getSlotPermit() { - return slotPermit; - } - - @Override - public SI getSlotInfo() { - return slotInfo; - } - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowPollTask.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowPollTask.java index 7035692b1..f1dcb60ec 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowPollTask.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowPollTask.java @@ -35,8 +35,8 @@ import io.temporal.serviceclient.MetricsTag; import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.worker.MetricsType; -import io.temporal.worker.tuning.*; import java.util.Objects; +import java.util.concurrent.Semaphore; import java.util.function.Supplier; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -46,7 +46,7 @@ final class WorkflowPollTask implements Poller.PollTask { private static final Logger log = LoggerFactory.getLogger(WorkflowPollTask.class); - private final TrackingSlotSupplier slotSupplier; + private final Semaphore workflowTaskExecutorSemaphore; private final StickyQueueBalancer stickyQueueBalancer; private final Scope metricsScope; private final Scope stickyMetricsScope; @@ -62,11 +62,11 @@ public WorkflowPollTask( @Nonnull String identity, @Nullable String buildId, boolean useBuildIdForVersioning, - @Nonnull TrackingSlotSupplier slotSupplier, + @Nonnull Semaphore workflowTaskExecutorSemaphore, @Nonnull StickyQueueBalancer stickyQueueBalancer, @Nonnull Scope workerMetricsScope, @Nonnull Supplier serverCapabilities) { - this.slotSupplier = Objects.requireNonNull(slotSupplier); + this.workflowTaskExecutorSemaphore = Objects.requireNonNull(workflowTaskExecutorSemaphore); this.stickyQueueBalancer = Objects.requireNonNull(stickyQueueBalancer); this.metricsScope = Objects.requireNonNull(workerMetricsScope); this.stickyMetricsScope = @@ -120,20 +120,11 @@ public WorkflowPollTask( @Override public WorkflowTask poll() { boolean isSuccessful = false; - SlotPermit permit; try { - permit = - slotSupplier.reserveSlot( - new SlotReservationData( - pollRequest.getTaskQueue().getName(), - pollRequest.getIdentity(), - pollRequest.getWorkerVersionCapabilities().getBuildId())); + workflowTaskExecutorSemaphore.acquire(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; - } catch (Exception e) { - log.warn("Error while trying to reserve a slot for workflow task", e.getCause()); - return null; } TaskQueueKind taskQueueKind = stickyQueueBalancer.makePoll(); @@ -149,12 +140,10 @@ public WorkflowTask poll() { } isSuccessful = true; stickyQueueBalancer.finishPoll(taskQueueKind, response.getBacklogCountHint()); - slotSupplier.markSlotUsed(new WorkflowSlotInfo(response, pollRequest), permit); - return new WorkflowTask( - response, () -> slotSupplier.releaseSlot(SlotReleaseReason.taskComplete(), permit)); + return new WorkflowTask(response, workflowTaskExecutorSemaphore::release); } finally { if (!isSuccessful) { - slotSupplier.releaseSlot(SlotReleaseReason.neverUsed(), permit); + workflowTaskExecutorSemaphore.release(); stickyQueueBalancer.finishPoll(taskQueueKind, 0); } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowWorker.java index 7070d6f16..42a9ae450 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/WorkflowWorker.java @@ -41,11 +41,11 @@ import io.temporal.worker.MetricsType; import io.temporal.worker.WorkerMetricsTag; import io.temporal.worker.WorkflowTaskDispatchHandle; -import io.temporal.worker.tuning.WorkflowSlotInfo; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -69,7 +69,8 @@ final class WorkflowWorker implements SuspendableWorker { private final Scope workerMetricsScope; private final GrpcRetryer grpcRetryer; private final EagerActivityDispatcher eagerActivityDispatcher; - private final TrackingSlotSupplier slotSupplier; + private final int executorSlots; + private final Semaphore executorSlotsSemaphore; private PollTaskExecutor pollTaskExecutor; @@ -88,8 +89,7 @@ public WorkflowWorker( @Nonnull WorkflowRunLockManager runLocks, @Nonnull WorkflowExecutorCache cache, @Nonnull WorkflowTaskHandler handler, - @Nonnull EagerActivityDispatcher eagerActivityDispatcher, - @Nonnull TrackingSlotSupplier slotSupplier) { + @Nonnull EagerActivityDispatcher eagerActivityDispatcher) { this.service = Objects.requireNonNull(service); this.namespace = Objects.requireNonNull(namespace); this.taskQueue = Objects.requireNonNull(taskQueue); @@ -98,13 +98,13 @@ public WorkflowWorker( this.pollerOptions = getPollerOptions(options); this.workerMetricsScope = MetricsTag.tagged(options.getMetricsScope(), WorkerMetricsTag.WorkerType.WORKFLOW_WORKER); - slotSupplier.setMetricsScope(workerMetricsScope); this.runLocks = Objects.requireNonNull(runLocks); this.cache = Objects.requireNonNull(cache); this.handler = Objects.requireNonNull(handler); this.grpcRetryer = new GrpcRetryer(service.getServerCapabilities()); this.eagerActivityDispatcher = eagerActivityDispatcher; - this.slotSupplier = slotSupplier; + this.executorSlots = options.getTaskExecutorThreadPoolSize(); + this.executorSlotsSemaphore = new Semaphore(executorSlots); } @Override @@ -117,7 +117,8 @@ public boolean start() { options.getIdentity(), new TaskHandlerImpl(handler), pollerOptions, - this.slotSupplier.maximumSlots(), + options.getTaskExecutorThreadPoolSize(), + workerMetricsScope, true); stickyQueueBalancer = new StickyQueueBalancer( @@ -134,7 +135,7 @@ public boolean start() { options.getIdentity(), options.getBuildId(), options.isUsingBuildIdForVersioning(), - slotSupplier, + executorSlotsSemaphore, stickyQueueBalancer, workerMetricsScope, service.getServerCapabilities()), @@ -153,7 +154,7 @@ public boolean start() { @Override public CompletableFuture shutdown(ShutdownManager shutdownManager, boolean interruptTasks) { - String supplierName = this + "#executorSlots"; + String semaphoreName = this + "#executorSlotsSemaphore"; boolean stickyQueueBalancerDrainEnabled = !interruptTasks @@ -172,8 +173,8 @@ public CompletableFuture shutdown(ShutdownManager shutdownManager, boolean .thenCompose( ignore -> !interruptTasks - ? shutdownManager.waitForSupplierPermitsReleasedUnlimited( - slotSupplier, supplierName) + ? shutdownManager.waitForSemaphorePermitsReleaseUntimed( + executorSlotsSemaphore, executorSlots, semaphoreName) : CompletableFuture.completedFuture(null)) .thenCompose( ignore -> @@ -239,40 +240,32 @@ private PollerOptions getPollerOptions(SingleWorkerOptions options) { @Nullable public WorkflowTaskDispatchHandle reserveWorkflowExecutor() { - // to avoid pollTaskExecutor becoming null inside the lambda, we cache it here + // to avoid pollTaskExecutor to become null inside the lambda, we are caching it here final PollTaskExecutor executor = pollTaskExecutor; - if (executor == null || isSuspended()) { - return null; - } - return slotSupplier - .tryReserveSlot( - new SlotReservationData(taskQueue, options.getIdentity(), options.getBuildId())) - .map( - slotPermit -> - new WorkflowTaskDispatchHandle( - workflowTask -> { - String queueName = - workflowTask.getResponse().getWorkflowExecutionTaskQueue().getName(); - TaskQueueKind queueKind = - workflowTask.getResponse().getWorkflowExecutionTaskQueue().getKind(); - Preconditions.checkArgument( - this.taskQueue.equals(queueName) - || TaskQueueKind.TASK_QUEUE_KIND_STICKY.equals(queueKind) - && this.stickyTaskQueueName.equals(queueName), - "Got a WFT for a wrong queue %s, expected %s or %s", - queueName, - this.taskQueue, - this.stickyTaskQueueName); - try { - pollTaskExecutor.process(workflowTask); - return true; - } catch (RejectedExecutionException e) { - return false; - } - }, - slotSupplier, - slotPermit)) - .orElse(null); + return executor != null && !isSuspended() && executorSlotsSemaphore.tryAcquire() + ? new WorkflowTaskDispatchHandle( + workflowTask -> { + String queueName = + workflowTask.getResponse().getWorkflowExecutionTaskQueue().getName(); + TaskQueueKind queueKind = + workflowTask.getResponse().getWorkflowExecutionTaskQueue().getKind(); + Preconditions.checkArgument( + this.taskQueue.equals(queueName) + || TaskQueueKind.TASK_QUEUE_KIND_STICKY.equals(queueKind) + && this.stickyTaskQueueName.equals(queueName), + "Got a WFT for a wrong queue %s, expected %s or %s", + queueName, + this.taskQueue, + this.stickyTaskQueueName); + try { + pollTaskExecutor.process(workflowTask); + return true; + } catch (RejectedExecutionException e) { + return false; + } + }, + executorSlotsSemaphore) + : null; } @Override diff --git a/temporal-sdk/src/main/java/io/temporal/worker/MetricsType.java b/temporal-sdk/src/main/java/io/temporal/worker/MetricsType.java index f58593c24..f3627beb8 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/MetricsType.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/MetricsType.java @@ -169,13 +169,4 @@ private MetricsType() {} // gauge public static final String WORKFLOW_ACTIVE_THREAD_COUNT = TEMPORAL_METRICS_PREFIX + "workflow_active_thread_count"; - - // - // Resource tuner - // - // Tagged with namespace & task_queue - public static final String RESOURCE_MEM_USAGE = "resource_slots_mem_usage"; - public static final String RESOURCE_CPU_USAGE = "resource_slots_cpu_usage"; - public static final String RESOURCE_MEM_PID = "resource_slots_mem_pid_output"; - public static final String RESOURCE_CPU_PID = "resource_slots_cpu_pid_output"; } diff --git a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java index 6c5d0740c..85ea85d71 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java @@ -39,7 +39,6 @@ import io.temporal.internal.worker.WorkflowExecutorCache; import io.temporal.serviceclient.MetricsTag; import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.worker.tuning.*; import io.temporal.workflow.Functions.Func; import io.temporal.workflow.WorkflowMethod; import java.time.Duration; @@ -66,6 +65,7 @@ public final class Worker { final SyncWorkflowWorker workflowWorker; final SyncActivityWorker activityWorker; private final AtomicBoolean started = new AtomicBoolean(); + private final Scope metricsScope; /** * Creates worker that connects to an instance of the Temporal Service. @@ -100,29 +100,20 @@ public final class Worker { String namespace = clientOptions.getNamespace(); Map tags = new ImmutableMap.Builder(1).put(MetricsTag.TASK_QUEUE, taskQueue).build(); - Scope taggedScope = metricsScope.tagged(tags); + this.metricsScope = metricsScope.tagged(tags); SingleWorkerOptions activityOptions = toActivityOptions( - factoryOptions, this.options, clientOptions, contextPropagators, taggedScope); + factoryOptions, this.options, clientOptions, contextPropagators, this.metricsScope); if (this.options.isLocalActivityWorkerOnly()) { activityWorker = null; } else { - TrackingSlotSupplier activitySlotSupplier = - new TrackingSlotSupplier<>( - this.options.getWorkerTuner() == null - ? new FixedSizeSlotSupplier<>( - this.options.getMaxConcurrentActivityExecutionSize()) - : this.options.getWorkerTuner().getActivityTaskSlotSupplier()); - activitySlotSupplier.attachMetricsToResourceController(taggedScope); - activityWorker = new SyncActivityWorker( service, namespace, taskQueue, this.options.getMaxTaskQueueActivitiesPerSecond(), - activityOptions, - activitySlotSupplier); + activityOptions); } EagerActivityDispatcher eagerActivityDispatcher = @@ -137,25 +128,10 @@ public final class Worker { clientOptions, taskQueue, contextPropagators, - taggedScope); + this.metricsScope); SingleWorkerOptions localActivityOptions = toLocalActivityOptions( - factoryOptions, this.options, clientOptions, contextPropagators, taggedScope); - - TrackingSlotSupplier workflowSlotSupplier = - new TrackingSlotSupplier<>( - this.options.getWorkerTuner() == null - ? new FixedSizeSlotSupplier<>( - this.options.getMaxConcurrentWorkflowTaskExecutionSize()) - : this.options.getWorkerTuner().getWorkflowTaskSlotSupplier()); - workflowSlotSupplier.attachMetricsToResourceController(taggedScope); - TrackingSlotSupplier localActivitySlotSupplier = - new TrackingSlotSupplier<>( - this.options.getWorkerTuner() == null - ? new FixedSizeSlotSupplier<>( - this.options.getMaxConcurrentLocalActivityExecutionSize()) - : this.options.getWorkerTuner().getLocalActivitySlotSupplier()); - localActivitySlotSupplier.attachMetricsToResourceController(taggedScope); + factoryOptions, this.options, clientOptions, contextPropagators, this.metricsScope); workflowWorker = new SyncWorkflowWorker( service, @@ -167,9 +143,7 @@ public final class Worker { cache, useStickyTaskQueue ? getStickyTaskQueueName(client.getOptions().getIdentity()) : null, workflowThreadExecutor, - eagerActivityDispatcher, - workflowSlotSupplier, - localActivitySlotSupplier); + eagerActivityDispatcher); } /** @@ -534,6 +508,7 @@ private static SingleWorkerOptions toActivityOptions( .setMaximumPollRatePerSecond(options.getMaxWorkerActivitiesPerSecond()) .setPollThreadCount(options.getMaxConcurrentActivityTaskPollers()) .build()) + .setTaskExecutorThreadPoolSize(options.getMaxConcurrentActivityExecutionSize()) .setMetricsScope(metricsScope) .build(); } @@ -566,6 +541,7 @@ private static SingleWorkerOptions toWorkflowWorkerOptions( return toSingleWorkerOptions(factoryOptions, options, clientOptions, contextPropagators) .setPollerOptions( PollerOptions.newBuilder().setPollThreadCount(maxConcurrentWorkflowTaskPollers).build()) + .setTaskExecutorThreadPoolSize(options.getMaxConcurrentWorkflowTaskExecutionSize()) .setStickyQueueScheduleToStartTimeout(stickyQueueScheduleToStartTimeout) .setStickyTaskQueueDrainTimeout(options.getStickyTaskQueueDrainTimeout()) .setDefaultDeadlockDetectionTimeout(options.getDefaultDeadlockDetectionTimeout()) @@ -581,6 +557,7 @@ private static SingleWorkerOptions toLocalActivityOptions( Scope metricsScope) { return toSingleWorkerOptions(factoryOptions, options, clientOptions, contextPropagators) .setPollerOptions(PollerOptions.newBuilder().setPollThreadCount(1).build()) + .setTaskExecutorThreadPoolSize(options.getMaxConcurrentLocalActivityExecutionSize()) .setMetricsScope(metricsScope) .build(); } diff --git a/temporal-sdk/src/main/java/io/temporal/worker/WorkerOptions.java b/temporal-sdk/src/main/java/io/temporal/worker/WorkerOptions.java index d9fce2e5e..52d5a75bf 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/WorkerOptions.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/WorkerOptions.java @@ -25,7 +25,6 @@ import com.google.common.base.Preconditions; import io.temporal.common.Experimental; import io.temporal.serviceclient.WorkflowServiceStubsOptions; -import io.temporal.worker.tuning.*; import java.time.Duration; import java.util.Objects; import javax.annotation.Nonnull; @@ -83,7 +82,6 @@ public static final class Builder { private String buildId; private boolean useBuildIdForVersioning; private Duration stickyTaskQueueDrainTimeout; - private WorkerTuner workerTuner; private String identity; private Builder() {} @@ -96,7 +94,6 @@ private Builder(WorkerOptions o) { this.maxConcurrentActivityExecutionSize = o.maxConcurrentActivityExecutionSize; this.maxConcurrentWorkflowTaskExecutionSize = o.maxConcurrentWorkflowTaskExecutionSize; this.maxConcurrentLocalActivityExecutionSize = o.maxConcurrentLocalActivityExecutionSize; - this.workerTuner = o.workerTuner; this.maxTaskQueueActivitiesPerSecond = o.maxTaskQueueActivitiesPerSecond; this.maxConcurrentWorkflowTaskPollers = o.maxConcurrentWorkflowTaskPollers; this.maxConcurrentActivityTaskPollers = o.maxConcurrentActivityTaskPollers; @@ -134,7 +131,6 @@ public Builder setMaxWorkerActivitiesPerSecond(double maxWorkerActivitiesPerSeco * @param maxConcurrentActivityExecutionSize Maximum number of activities executed in parallel. * Default is 200, which is chosen if set to zero. * @return {@code this} - *

Note setting is mutually exclusive with {@link #setWorkerTuner(WorkerTuner)} */ public Builder setMaxConcurrentActivityExecutionSize(int maxConcurrentActivityExecutionSize) { if (maxConcurrentActivityExecutionSize < 0) { @@ -152,7 +148,6 @@ public Builder setMaxConcurrentActivityExecutionSize(int maxConcurrentActivityEx * @return {@code this} *

Note that this is not related to the total number of open workflows which do not need * to be loaded in a worker when they are not making state transitions. - *

Note setting is mutually exclusive with {@link #setWorkerTuner(WorkerTuner)} */ public Builder setMaxConcurrentWorkflowTaskExecutionSize( int maxConcurrentWorkflowTaskExecutionSize) { @@ -169,7 +164,6 @@ public Builder setMaxConcurrentWorkflowTaskExecutionSize( * @param maxConcurrentLocalActivityExecutionSize Maximum number of local activities executed in * parallel. Default is 200, which is chosen if set to zero. * @return {@code this} - *

Note setting is mutually exclusive with {@link #setWorkerTuner(WorkerTuner)} */ public Builder setMaxConcurrentLocalActivityExecutionSize( int maxConcurrentLocalActivityExecutionSize) { @@ -377,16 +371,6 @@ public Builder setStickyTaskQueueDrainTimeout(Duration stickyTaskQueueDrainTimeo return this; } - /** - * Set a {@link WorkerTuner} to determine how slots will be allocated for different types of - * tasks. - */ - @Experimental - public Builder setWorkerTuner(WorkerTuner workerTuner) { - this.workerTuner = workerTuner; - return this; - } - /** Override identity of the worker primary specified in a WorkflowClient options. */ public Builder setIdentity(String identity) { this.identity = identity; @@ -399,7 +383,6 @@ public WorkerOptions build() { maxConcurrentActivityExecutionSize, maxConcurrentWorkflowTaskExecutionSize, maxConcurrentLocalActivityExecutionSize, - workerTuner, maxTaskQueueActivitiesPerSecond, maxConcurrentWorkflowTaskPollers, maxConcurrentActivityTaskPollers, @@ -426,21 +409,6 @@ public WorkerOptions validateAndBuildWithDefaults() { Preconditions.checkState( maxConcurrentLocalActivityExecutionSize >= 0, "negative maxConcurrentLocalActivityExecutionSize"); - if (workerTuner != null) { - Preconditions.checkState( - maxConcurrentActivityExecutionSize == 0, - "maxConcurrentActivityExecutionSize must not be set if workerTuner is set"); - } - if (workerTuner != null) { - Preconditions.checkState( - maxConcurrentWorkflowTaskExecutionSize == 0, - "maxConcurrentWorkflowTaskExecutionSize must not be set if workerTuner is set"); - } - if (workerTuner != null) { - Preconditions.checkState( - maxConcurrentLocalActivityExecutionSize == 0, - "maxConcurrentLocalActivityExecutionSize must not be set if workerTuner is set"); - } Preconditions.checkState( maxTaskQueueActivitiesPerSecond >= 0, "negative taskQueueActivitiesPerSecond"); Preconditions.checkState( @@ -473,7 +441,6 @@ public WorkerOptions validateAndBuildWithDefaults() { maxConcurrentLocalActivityExecutionSize == 0 ? DEFAULT_MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE : maxConcurrentLocalActivityExecutionSize, - workerTuner, maxTaskQueueActivitiesPerSecond, maxConcurrentWorkflowTaskPollers == 0 ? DEFAULT_MAX_CONCURRENT_WORKFLOW_TASK_POLLERS @@ -508,7 +475,6 @@ public WorkerOptions validateAndBuildWithDefaults() { private final int maxConcurrentActivityExecutionSize; private final int maxConcurrentWorkflowTaskExecutionSize; private final int maxConcurrentLocalActivityExecutionSize; - private final WorkerTuner workerTuner; private final double maxTaskQueueActivitiesPerSecond; private final int maxConcurrentWorkflowTaskPollers; private final int maxConcurrentActivityTaskPollers; @@ -526,9 +492,8 @@ public WorkerOptions validateAndBuildWithDefaults() { private WorkerOptions( double maxWorkerActivitiesPerSecond, int maxConcurrentActivityExecutionSize, - int maxConcurrentWorkflowTaskExecutionSize, + int maxConcurrentWorkflowExecutionSize, int maxConcurrentLocalActivityExecutionSize, - WorkerTuner workerTuner, double maxTaskQueueActivitiesPerSecond, int workflowPollThreadCount, int activityPollThreadCount, @@ -544,9 +509,8 @@ private WorkerOptions( String identity) { this.maxWorkerActivitiesPerSecond = maxWorkerActivitiesPerSecond; this.maxConcurrentActivityExecutionSize = maxConcurrentActivityExecutionSize; - this.maxConcurrentWorkflowTaskExecutionSize = maxConcurrentWorkflowTaskExecutionSize; + this.maxConcurrentWorkflowTaskExecutionSize = maxConcurrentWorkflowExecutionSize; this.maxConcurrentLocalActivityExecutionSize = maxConcurrentLocalActivityExecutionSize; - this.workerTuner = workerTuner; this.maxTaskQueueActivitiesPerSecond = maxTaskQueueActivitiesPerSecond; this.maxConcurrentWorkflowTaskPollers = workflowPollThreadCount; this.maxConcurrentActivityTaskPollers = activityPollThreadCount; @@ -643,10 +607,6 @@ public Duration getStickyTaskQueueDrainTimeout() { return stickyTaskQueueDrainTimeout; } - public WorkerTuner getWorkerTuner() { - return workerTuner; - } - @Nullable public String getIdentity() { return identity; @@ -657,22 +617,21 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; WorkerOptions that = (WorkerOptions) o; - return compare(maxWorkerActivitiesPerSecond, that.maxWorkerActivitiesPerSecond) == 0 + return compare(that.maxWorkerActivitiesPerSecond, maxWorkerActivitiesPerSecond) == 0 && maxConcurrentActivityExecutionSize == that.maxConcurrentActivityExecutionSize && maxConcurrentWorkflowTaskExecutionSize == that.maxConcurrentWorkflowTaskExecutionSize && maxConcurrentLocalActivityExecutionSize == that.maxConcurrentLocalActivityExecutionSize - && compare(maxTaskQueueActivitiesPerSecond, that.maxTaskQueueActivitiesPerSecond) == 0 + && compare(that.maxTaskQueueActivitiesPerSecond, maxTaskQueueActivitiesPerSecond) == 0 && maxConcurrentWorkflowTaskPollers == that.maxConcurrentWorkflowTaskPollers && maxConcurrentActivityTaskPollers == that.maxConcurrentActivityTaskPollers && localActivityWorkerOnly == that.localActivityWorkerOnly && defaultDeadlockDetectionTimeout == that.defaultDeadlockDetectionTimeout - && disableEagerExecution == that.disableEagerExecution - && useBuildIdForVersioning == that.useBuildIdForVersioning - && Objects.equals(workerTuner, that.workerTuner) && Objects.equals(maxHeartbeatThrottleInterval, that.maxHeartbeatThrottleInterval) && Objects.equals(defaultHeartbeatThrottleInterval, that.defaultHeartbeatThrottleInterval) && Objects.equals(stickyQueueScheduleToStartTimeout, that.stickyQueueScheduleToStartTimeout) - && Objects.equals(buildId, that.buildId) + && disableEagerExecution == that.disableEagerExecution + && useBuildIdForVersioning == that.useBuildIdForVersioning + && Objects.equals(that.buildId, buildId) && Objects.equals(stickyTaskQueueDrainTimeout, that.stickyTaskQueueDrainTimeout) && Objects.equals(identity, that.identity); } @@ -684,7 +643,6 @@ public int hashCode() { maxConcurrentActivityExecutionSize, maxConcurrentWorkflowTaskExecutionSize, maxConcurrentLocalActivityExecutionSize, - workerTuner, maxTaskQueueActivitiesPerSecond, maxConcurrentWorkflowTaskPollers, maxConcurrentActivityTaskPollers, @@ -711,8 +669,6 @@ public String toString() { + maxConcurrentWorkflowTaskExecutionSize + ", maxConcurrentLocalActivityExecutionSize=" + maxConcurrentLocalActivityExecutionSize - + ", workerTuner=" - + workerTuner + ", maxTaskQueueActivitiesPerSecond=" + maxTaskQueueActivitiesPerSecond + ", maxConcurrentWorkflowTaskPollers=" @@ -735,8 +691,7 @@ public String toString() { + useBuildIdForVersioning + ", buildId='" + buildId - + '\'' - + ", stickyTaskQueueDrainTimeout=" + + ", stickyTaskQueueDrainTimeout='" + stickyTaskQueueDrainTimeout + ", identity=" + identity diff --git a/temporal-sdk/src/main/java/io/temporal/worker/WorkflowTaskDispatchHandle.java b/temporal-sdk/src/main/java/io/temporal/worker/WorkflowTaskDispatchHandle.java index 6ccfaa054..9189d6566 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/WorkflowTaskDispatchHandle.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/WorkflowTaskDispatchHandle.java @@ -22,12 +22,9 @@ import com.google.common.base.Preconditions; import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse; -import io.temporal.internal.worker.TrackingSlotSupplier; import io.temporal.internal.worker.WorkflowTask; -import io.temporal.worker.tuning.SlotPermit; -import io.temporal.worker.tuning.SlotReleaseReason; -import io.temporal.worker.tuning.WorkflowSlotInfo; import java.io.Closeable; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import javax.annotation.Nonnull; @@ -35,20 +32,17 @@ public class WorkflowTaskDispatchHandle implements Closeable { private final AtomicBoolean completed = new AtomicBoolean(); private final Function dispatchCallback; - private final TrackingSlotSupplier slotSupplier; - private final SlotPermit permit; + private final Semaphore executorSlotsSemaphore; /** * @param dispatchCallback callback into a {@code WorkflowWorker} to dispatch a workflow task. - * @param slotSupplier slot supplier that was used to reserve a slot for this workflow task + * @param executorSlotsSemaphore worker executor slots semaphore that was used to reserve this + * dispatch handle on */ public WorkflowTaskDispatchHandle( - DispatchCallback dispatchCallback, - TrackingSlotSupplier slotSupplier, - SlotPermit permit) { + DispatchCallback dispatchCallback, Semaphore executorSlotsSemaphore) { this.dispatchCallback = dispatchCallback; - this.slotSupplier = slotSupplier; - this.permit = permit; + this.executorSlotsSemaphore = executorSlotsSemaphore; } /** @@ -61,9 +55,7 @@ public boolean dispatch(@Nonnull PollWorkflowTaskQueueResponse workflowTask) { Preconditions.checkNotNull(workflowTask, "workflowTask"); if (completed.compareAndSet(false, true)) { return dispatchCallback.apply( - new WorkflowTask( - workflowTask, - () -> slotSupplier.releaseSlot(SlotReleaseReason.taskComplete(), permit))); + new WorkflowTask(workflowTask, executorSlotsSemaphore::release)); } else { return false; } @@ -72,7 +64,7 @@ public boolean dispatch(@Nonnull PollWorkflowTaskQueueResponse workflowTask) { @Override public void close() { if (completed.compareAndSet(false, true)) { - slotSupplier.releaseSlot(SlotReleaseReason.neverUsed(), permit); + executorSlotsSemaphore.release(); } } diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ActivitySlotInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ActivitySlotInfo.java deleted file mode 100644 index 2a86e4a65..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ActivitySlotInfo.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.activity.ActivityInfo; -import io.temporal.common.Experimental; -import java.util.Objects; - -/** Contains information about a slot that is being used to execute an activity task. */ -@Experimental -public class ActivitySlotInfo extends SlotInfo { - private final ActivityInfo activityInfo; - private final String workerIdentity; - private final String workerBuildId; - - public ActivitySlotInfo(ActivityInfo activityInfo, String workerIdentity, String workerBuildId) { - this.activityInfo = activityInfo; - this.workerIdentity = workerIdentity; - this.workerBuildId = workerBuildId; - } - - public ActivityInfo getActivityInfo() { - return activityInfo; - } - - public String getWorkerIdentity() { - return workerIdentity; - } - - public String getWorkerBuildId() { - return workerBuildId; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ActivitySlotInfo that = (ActivitySlotInfo) o; - return Objects.equals(activityInfo, that.activityInfo) - && Objects.equals(workerIdentity, that.workerIdentity) - && Objects.equals(workerBuildId, that.workerBuildId); - } - - @Override - public int hashCode() { - return Objects.hash(activityInfo, workerIdentity, workerBuildId); - } - - @Override - public String toString() { - return "ActivitySlotInfo{" - + "activityInfo=" - + activityInfo - + ", workerIdentity='" - + workerIdentity - + '\'' - + ", workerBuildId='" - + workerBuildId - + '\'' - + '}'; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/CompositeTuner.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/CompositeTuner.java deleted file mode 100644 index 679050493..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/CompositeTuner.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.util.Objects; -import javax.annotation.Nonnull; - -/** - * Can be used to create a {@link WorkerTuner} which uses specific {@link SlotSupplier}s for each - * type of slot. - */ -@Experimental -public class CompositeTuner implements WorkerTuner { - private final @Nonnull SlotSupplier workflowTaskSlotSupplier; - private final @Nonnull SlotSupplier activityTaskSlotSupplier; - private final @Nonnull SlotSupplier localActivitySlotSupplier; - - public CompositeTuner( - @Nonnull SlotSupplier workflowTaskSlotSupplier, - @Nonnull SlotSupplier activityTaskSlotSupplier, - @Nonnull SlotSupplier localActivitySlotSupplier) { - this.workflowTaskSlotSupplier = Objects.requireNonNull(workflowTaskSlotSupplier); - this.activityTaskSlotSupplier = Objects.requireNonNull(activityTaskSlotSupplier); - this.localActivitySlotSupplier = Objects.requireNonNull(localActivitySlotSupplier); - - // All resource-based slot suppliers must use the same controller - validateResourceController(workflowTaskSlotSupplier, activityTaskSlotSupplier); - validateResourceController(workflowTaskSlotSupplier, localActivitySlotSupplier); - validateResourceController(activityTaskSlotSupplier, localActivitySlotSupplier); - } - - @Nonnull - @Override - public SlotSupplier getWorkflowTaskSlotSupplier() { - return workflowTaskSlotSupplier; - } - - @Nonnull - @Override - public SlotSupplier getActivityTaskSlotSupplier() { - return activityTaskSlotSupplier; - } - - @Nonnull - @Override - public SlotSupplier getLocalActivitySlotSupplier() { - return localActivitySlotSupplier; - } - - private void validateResourceController( - @Nonnull SlotSupplier supplier1, @Nonnull SlotSupplier supplier2) { - if (supplier1 instanceof ResourceBasedSlotSupplier - && supplier2 instanceof ResourceBasedSlotSupplier) { - ResourceBasedController controller1 = - ((ResourceBasedSlotSupplier) supplier1).getResourceController(); - ResourceBasedController controller2 = - ((ResourceBasedSlotSupplier) supplier2).getResourceController(); - if (controller1 != controller2) { - throw new IllegalArgumentException( - "All resource-based slot suppliers must use the same ResourceController"); - } - } - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/FixedSizeSlotSupplier.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/FixedSizeSlotSupplier.java deleted file mode 100644 index 870102ff6..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/FixedSizeSlotSupplier.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import com.google.common.base.Preconditions; -import java.util.Optional; -import java.util.concurrent.*; - -/** - * This implementation of {@link SlotSupplier} provides a fixed number of slots backed by a - * semaphore, and is the default behavior when a custom supplier is not explicitly specified. - * - * @param The slot info type for this supplier. - */ -public class FixedSizeSlotSupplier implements SlotSupplier { - private final int numSlots; - private final Semaphore executorSlotsSemaphore; - - public FixedSizeSlotSupplier(int numSlots) { - Preconditions.checkArgument(numSlots > 0, "FixedSizeSlotSupplier must have at least one slot"); - this.numSlots = numSlots; - executorSlotsSemaphore = new Semaphore(numSlots); - } - - @Override - public SlotPermit reserveSlot(SlotReserveContext ctx) throws InterruptedException { - executorSlotsSemaphore.acquire(); - return new SlotPermit(); - } - - @Override - public Optional tryReserveSlot(SlotReserveContext ctx) { - boolean gotOne = executorSlotsSemaphore.tryAcquire(); - if (gotOne) { - return Optional.of(new SlotPermit()); - } - return Optional.empty(); - } - - @Override - public void markSlotUsed(SlotMarkUsedContext ctx) {} - - @Override - public void releaseSlot(SlotReleaseContext ctx) { - executorSlotsSemaphore.release(); - } - - @Override - public int getMaximumSlots() { - return numSlots; - } - - @Override - public String toString() { - return "FixedSizeSlotSupplier{" + "numSlots=" + numSlots + '}'; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/JVMSystemResourceInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/JVMSystemResourceInfo.java deleted file mode 100644 index ca20d2473..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/JVMSystemResourceInfo.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import com.sun.management.OperatingSystemMXBean; -import io.temporal.common.Experimental; -import java.lang.management.ManagementFactory; -import java.time.Instant; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -/** {@link SystemResourceInfo} implementation that uses JVM-specific APIs to get resource usage. */ -@Experimental -public class JVMSystemResourceInfo implements SystemResourceInfo { - // As of relatively recent Java versions (including backports), this class will properly deal with - // containerized environments as well as running on bare metal. - // See https://bugs.openjdk.org/browse/JDK-8226575 for more details on which versions the fixes - // have been backported to. - OperatingSystemMXBean osBean = ManagementFactory.getPlatformMXBean(OperatingSystemMXBean.class); - - private final Lock refreshLock = new ReentrantLock(); - private SystemInfo lastSystemInfo; - - @Override - public double getCPUUsagePercent() { - return refresh().cpuUsagePercent; - } - - @Override - public double getMemoryUsagePercent() { - return refresh().memoryUsagePercent; - } - - @SuppressWarnings("deprecation") // deprecated APIs needed since replacements are for Java 14+ - private SystemInfo refresh() { - - refreshLock.lock(); - try { - if (lastSystemInfo == null - || Instant.now().isAfter(lastSystemInfo.refreshed.plusMillis(100))) { - // This can return NaN seemingly when usage is very low - double lastCpuUsage = osBean.getSystemCpuLoad(); - if (lastCpuUsage < 0 || Double.isNaN(lastCpuUsage)) { - lastCpuUsage = 0; - } - - Runtime runtime = Runtime.getRuntime(); - long jvmUsedMemory = runtime.totalMemory() - runtime.freeMemory(); - long jvmMaxMemory = runtime.maxMemory(); - - double lastMemUsage = ((double) jvmUsedMemory / jvmMaxMemory); - Instant lastRefresh = Instant.now(); - lastSystemInfo = new SystemInfo(lastRefresh, lastCpuUsage, lastMemUsage); - } - } finally { - refreshLock.unlock(); - } - - return lastSystemInfo; - } - - private static class SystemInfo { - private final Instant refreshed; - private final double cpuUsagePercent; - private final double memoryUsagePercent; - - private SystemInfo(Instant refreshed, double cpuUsagePercent, double memoryUsagePercent) { - this.refreshed = refreshed; - this.cpuUsagePercent = cpuUsagePercent; - this.memoryUsagePercent = memoryUsagePercent; - } - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/LocalActivitySlotInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/LocalActivitySlotInfo.java deleted file mode 100644 index 6e2235866..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/LocalActivitySlotInfo.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.activity.ActivityInfo; -import io.temporal.common.Experimental; -import java.util.Objects; - -/** Contains information about a slot that is being used to execute a local activity. */ -@Experimental -public class LocalActivitySlotInfo extends SlotInfo { - private final ActivityInfo activityInfo; - private final String workerIdentity; - private final String workerBuildId; - - public LocalActivitySlotInfo(ActivityInfo info, String workerIdentity, String workerBuildId) { - this.activityInfo = info; - this.workerIdentity = workerIdentity; - this.workerBuildId = workerBuildId; - } - - public ActivityInfo getActivityInfo() { - return activityInfo; - } - - public String getWorkerIdentity() { - return workerIdentity; - } - - public String getWorkerBuildId() { - return workerBuildId; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - LocalActivitySlotInfo that = (LocalActivitySlotInfo) o; - return Objects.equals(activityInfo, that.activityInfo) - && Objects.equals(workerIdentity, that.workerIdentity) - && Objects.equals(workerBuildId, that.workerBuildId); - } - - @Override - public int hashCode() { - return Objects.hash(activityInfo, workerIdentity, workerBuildId); - } - - @Override - public String toString() { - return "LocalActivitySlotInfo{" - + "activityInfo=" - + activityInfo - + ", workerIdentity='" - + workerIdentity - + '\'' - + ", workerBuildId='" - + workerBuildId - + '\'' - + '}'; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/PIDController.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/PIDController.java deleted file mode 100644 index d9135bae2..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/PIDController.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -/** - * A simple PID closed control loop.
- *
- * License : MIT - * - * @author Charles Grassin - */ -class PIDController { - // PID coefficients - private double setPoint; - private double kP, kI, kD; - - /** Limit bound of the output. */ - private double minLimit = Double.NaN, maxLimit = Double.NaN; - - // Dynamic variables - private double previousTime = Double.NaN; - private double lastError = 0; - private double integralError = 0; - - /** - * Constructs a new PID with set coefficients. - * - * @param setPoint The initial target value. - * @param kP The proportional gain coefficient. - * @param kI The integral gain coefficient. - * @param kD The derivative gain coefficient. - */ - PIDController(final double setPoint, final double kP, final double kI, final double kD) { - this.setSetpoint(setPoint); - this.kP = kP; - this.kI = kI; - this.kD = kD; - } - - /** - * Updates the controller with the current time and value and outputs the PID controller output. - * - * @param currentTime The current time (in arbitrary time unit, such as seconds). If the PID is - * assumed to run at a constant frequency, you can simply put '1'. - * @param currentValue The current, measured value. - * @return The PID controller output. - */ - double getOutput(final double currentTime, final double currentValue) { - final double error = setPoint - currentValue; - final double dt = (!Double.isNaN(previousTime)) ? (currentTime - previousTime) : 0; - - // Compute Integral & Derivative error - final double derivativeError = (dt != 0) ? ((error - lastError) / dt) : 0; - integralError += error * dt; - - // Save history - previousTime = currentTime; - lastError = error; - - return checkLimits((kP * error) + (kI * integralError) + (kD * derivativeError)); - } - - /** Resets the integral and derivative errors. */ - void reset() { - previousTime = 0; - lastError = 0; - integralError = 0; - } - - /** - * Bounds the PID output between the lower limit and the upper limit. - * - * @param output The target output value. - * @return The output value, bounded to the limits. - */ - private double checkLimits(final double output) { - if (!Double.isNaN(minLimit) && output < minLimit) return minLimit; - else if (!Double.isNaN(maxLimit) && output > maxLimit) return maxLimit; - else return output; - } - - // Getters & Setters - - /** - * Sets the output limits of the PID controller. If the minLimit is superior to the maxLimit, it - * will use the smallest as the minLimit. - * - * @param minLimit The lower limit of the PID output. - * @param maxLimit The upper limit of the PID output. - */ - void setOuputLimits(final double minLimit, final double maxLimit) { - if (minLimit < maxLimit) { - this.minLimit = minLimit; - this.maxLimit = maxLimit; - } else { - this.minLimit = maxLimit; - this.maxLimit = minLimit; - } - } - - /** Removes the output limits of the PID controller */ - void removeOuputLimits() { - this.minLimit = Double.NaN; - this.maxLimit = Double.NaN; - } - - /** - * @return the kP parameter - */ - public double getkP() { - return kP; - } - - /** - * @param kP the kP parameter to set - */ - void setkP(double kP) { - this.kP = kP; - reset(); - } - - /** - * @return the kI parameter - */ - double getkI() { - return kI; - } - - /** - * @param kI the kI parameter to set - */ - void setkI(double kI) { - this.kI = kI; - reset(); - } - - /** - * @return the kD parameter - */ - double getkD() { - return kD; - } - - /** - * @param kD the kD parameter to set - */ - void setkD(double kD) { - this.kD = kD; - reset(); - } - - /** - * @return the setPoint - */ - double getSetPoint() { - return setPoint; - } - - /** - * Establishes a new set point for the PID controller. - * - * @param setPoint The new target point. - */ - void setSetpoint(final double setPoint) { - reset(); - this.setPoint = setPoint; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedController.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedController.java deleted file mode 100644 index 111df0de3..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedController.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import com.uber.m3.tally.Gauge; -import com.uber.m3.tally.Scope; -import io.temporal.common.Experimental; -import io.temporal.worker.MetricsType; -import java.time.Instant; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; - -/** - * Is used by {@link ResourceBasedSlotSupplier} and {@link ResourceBasedTuner} to make decisions - * about whether slots should be handed out based on system resource usage. - */ -@Experimental -public class ResourceBasedController { - public final ResourceBasedControllerOptions options; - - private final ReentrantLock decisionLock = new ReentrantLock(); - private final PIDController memoryController; - private final PIDController cpuController; - private final SystemResourceInfo systemInfoSupplier; - private Instant lastPidRefresh = Instant.now(); - - private final AtomicReference metrics = new AtomicReference<>(); - - /** - * Construct a controller with the given options. If you want to use resource-based tuning for all - * slot suppliers, prefer {@link ResourceBasedTuner}. - */ - public static ResourceBasedController newSystemInfoController( - ResourceBasedControllerOptions options) { - return new ResourceBasedController(options, new JVMSystemResourceInfo()); - } - - /** - * Construct a controller with the given options and system info supplier. Users should prefer - * {@link #newSystemInfoController(ResourceBasedControllerOptions)}. - */ - public ResourceBasedController( - ResourceBasedControllerOptions options, SystemResourceInfo systemInfoSupplier) { - this.options = options; - this.systemInfoSupplier = systemInfoSupplier; - this.memoryController = - new PIDController( - options.getTargetCPUUsage(), - options.getMemoryPGain(), - options.getMemoryIGain(), - options.getMemoryDGain()); - this.cpuController = - new PIDController( - options.getTargetCPUUsage(), - options.getCpuPGain(), - options.getCpuIGain(), - options.getCpuDGain()); - } - - /** - * @return True if the PID controllers & and other constraints would allow another slot - */ - boolean pidDecision() { - decisionLock.lock(); - try { - double memoryUsage = systemInfoSupplier.getMemoryUsagePercent(); - double cpuUsage = systemInfoSupplier.getCPUUsagePercent(); - double memoryOutput = - memoryController.getOutput(lastPidRefresh.getEpochSecond(), memoryUsage); - double cpuOutput = cpuController.getOutput(lastPidRefresh.getEpochSecond(), cpuUsage); - lastPidRefresh = Instant.now(); - - Metrics metrics = this.metrics.get(); - if (metrics != null) { - metrics.memUsage.update(memoryUsage); - metrics.cpuUsage.update(cpuUsage); - metrics.memPidOut.update(memoryOutput); - metrics.cpuPidOut.update(cpuOutput); - } - - return memoryOutput > options.getMemoryOutputThreshold() - && cpuOutput > options.getCpuOutputThreshold() - && canReserve(); - } finally { - decisionLock.unlock(); - } - } - - private boolean canReserve() { - return systemInfoSupplier.getMemoryUsagePercent() < options.getTargetMemoryUsage(); - } - - /** Visible for internal usage. Can only be set once. */ - public void setMetricsScope(Scope metricsScope) { - if (metrics.get() == null) { - metrics.set(new Metrics(metricsScope)); - } - } - - private static class Metrics { - private final Gauge memUsage; - private final Gauge cpuUsage; - private final Gauge memPidOut; - private final Gauge cpuPidOut; - - private Metrics(Scope scope) { - memUsage = scope.gauge(MetricsType.RESOURCE_MEM_USAGE); - cpuUsage = scope.gauge(MetricsType.RESOURCE_CPU_USAGE); - memPidOut = scope.gauge(MetricsType.RESOURCE_MEM_PID); - cpuPidOut = scope.gauge(MetricsType.RESOURCE_CPU_PID); - } - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedControllerOptions.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedControllerOptions.java deleted file mode 100644 index 47ca10531..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedControllerOptions.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import com.google.common.base.Preconditions; -import io.temporal.common.Experimental; - -/** Options for a {@link ResourceBasedController} */ -@Experimental -public class ResourceBasedControllerOptions { - - public static ResourceBasedControllerOptions.Builder newBuilder( - double targetMemoryUsage, double targetCPUUsage) { - return new ResourceBasedControllerOptions.Builder() - .setTargetMemoryUsage(targetMemoryUsage) - .setTargetCPUUsage(targetCPUUsage); - } - - public static final class Builder { - private double targetMemoryUsage; - private double targetCPUUsage; - private double memoryPGain = 5; - private double memoryIGain = 0; - private double memoryDGain = 1; - private double memoryOutputThreshold = 0.25; - private double cpuPGain = 5; - private double cpuIGain = 0; - private double cpuDGain = 1; - private double cpuOutputThreshold = 0.05; - - public Builder setTargetMemoryUsage(double targetMemoryUsage) { - this.targetMemoryUsage = targetMemoryUsage; - return this; - } - - public Builder setTargetCPUUsage(double targetCPUUsage) { - this.targetCPUUsage = targetCPUUsage; - return this; - } - - public Builder setMemoryPGain(double memoryPGain) { - this.memoryPGain = memoryPGain; - return this; - } - - public Builder setMemoryIGain(double memoryIGain) { - this.memoryIGain = memoryIGain; - return this; - } - - public Builder setMemoryDGain(double memoryDGain) { - this.memoryDGain = memoryDGain; - return this; - } - - public Builder setMemoryOutputThreshold(double memoryOutputThreshold) { - this.memoryOutputThreshold = memoryOutputThreshold; - return this; - } - - public Builder setCpuPGain(double cpuPGain) { - this.cpuPGain = cpuPGain; - return this; - } - - public Builder setCpuIGain(double cpuIGain) { - this.cpuIGain = cpuIGain; - return this; - } - - public Builder setCpuDGain(double cpuDGain) { - this.cpuDGain = cpuDGain; - return this; - } - - public Builder setCpuOutputThreshold(double cpuOutputThreshold) { - this.cpuOutputThreshold = cpuOutputThreshold; - return this; - } - - public ResourceBasedControllerOptions build() { - Preconditions.checkState( - targetMemoryUsage > 0, "targetMemoryUsage must be set and greater than 0"); - Preconditions.checkState(targetCPUUsage > 0, "targetCPUUsage must be set and greater than 0"); - return new ResourceBasedControllerOptions(this); - } - } - - private final double targetMemoryUsage; - private final double targetCPUUsage; - - private final double memoryPGain; - private final double memoryIGain; - private final double memoryDGain; - private final double memoryOutputThreshold; - - private final double cpuPGain; - private final double cpuIGain; - private final double cpuDGain; - private final double cpuOutputThreshold; - - private ResourceBasedControllerOptions(Builder builder) { - this.targetMemoryUsage = builder.targetMemoryUsage; - this.targetCPUUsage = builder.targetCPUUsage; - this.memoryPGain = builder.memoryPGain; - this.memoryIGain = builder.memoryIGain; - this.memoryDGain = builder.memoryDGain; - this.memoryOutputThreshold = builder.memoryOutputThreshold; - this.cpuPGain = builder.cpuPGain; - this.cpuIGain = builder.cpuIGain; - this.cpuDGain = builder.cpuDGain; - this.cpuOutputThreshold = builder.cpuOutputThreshold; - } - - public double getTargetMemoryUsage() { - return targetMemoryUsage; - } - - public double getTargetCPUUsage() { - return targetCPUUsage; - } - - public double getMemoryPGain() { - return memoryPGain; - } - - public double getMemoryIGain() { - return memoryIGain; - } - - public double getMemoryDGain() { - return memoryDGain; - } - - public double getMemoryOutputThreshold() { - return memoryOutputThreshold; - } - - public double getCpuPGain() { - return cpuPGain; - } - - public double getCpuIGain() { - return cpuIGain; - } - - public double getCpuDGain() { - return cpuDGain; - } - - public double getCpuOutputThreshold() { - return cpuOutputThreshold; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotOptions.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotOptions.java deleted file mode 100644 index 3dad1ea39..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotOptions.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.time.Duration; -import java.util.Objects; - -/** Options resource-based slot suppliers */ -@Experimental -public class ResourceBasedSlotOptions { - private final int minimumSlots; - private final int maximumSlots; - private final Duration rampThrottle; - - public static Builder newBuilder() { - return new Builder(); - } - - public static final class Builder { - private int minimumSlots; - private int maximumSlots; - private Duration rampThrottle; - - private Builder() {} - - /** - * @param minimumSlots minimum number of slots that will be issued without any resource checks - */ - public Builder setMinimumSlots(int minimumSlots) { - this.minimumSlots = minimumSlots; - return this; - } - - /** - * @param maximumSlots maximum number of slots that will ever be issued - */ - public Builder setMaximumSlots(int maximumSlots) { - this.maximumSlots = maximumSlots; - return this; - } - - /** - * @param rampThrottle time to wait between slot issuance. This value matters because how many - * resources a task will use cannot be determined ahead of time, and thus the system should - * wait to see how much resources are used before issuing more slots. - */ - public Builder setRampThrottle(Duration rampThrottle) { - this.rampThrottle = rampThrottle; - return this; - } - - public ResourceBasedSlotOptions build() { - return new ResourceBasedSlotOptions(minimumSlots, maximumSlots, rampThrottle); - } - } - - /** - * @param minimumSlots minimum number of slots that will be issued without any resource checks - * @param maximumSlots maximum number of slots that will ever be issued - * @param rampThrottle time to wait between slot issuance. This value matters because how many - * resources a task will use cannot be determined ahead of time, and thus the system should - * wait to see how much resources are used before issuing more slots. - */ - private ResourceBasedSlotOptions(int minimumSlots, int maximumSlots, Duration rampThrottle) { - this.minimumSlots = minimumSlots; - this.maximumSlots = maximumSlots; - this.rampThrottle = rampThrottle; - } - - public int getMinimumSlots() { - return minimumSlots; - } - - public int getMaximumSlots() { - return maximumSlots; - } - - public Duration getRampThrottle() { - return rampThrottle; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ResourceBasedSlotOptions that = (ResourceBasedSlotOptions) o; - return minimumSlots == that.minimumSlots - && maximumSlots == that.maximumSlots - && Objects.equals(rampThrottle, that.rampThrottle); - } - - @Override - public int hashCode() { - return Objects.hash(minimumSlots, maximumSlots, rampThrottle); - } - - @Override - public String toString() { - return "ResourceBasedSlotOptions{" - + "minimumSlots=" - + minimumSlots - + ", maximumSlots=" - + maximumSlots - + ", rampThrottle=" - + rampThrottle - + '}'; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotSupplier.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotSupplier.java deleted file mode 100644 index b01ed9dbb..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedSlotSupplier.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.time.Duration; -import java.time.Instant; -import java.util.Optional; - -/** Implements a {@link SlotSupplier} based on resource usage for a particular slot type. */ -@Experimental -public class ResourceBasedSlotSupplier implements SlotSupplier { - - private final ResourceBasedController resourceController; - private final ResourceBasedSlotOptions options; - private Instant lastSlotIssuedAt = Instant.EPOCH; - - /** - * Construct a slot supplier for workflow tasks with the given resource controller and options. - * - *

The resource controller must be the same among all slot suppliers in a worker. If you want - * to use resource-based tuning for all slot suppliers, prefer {@link ResourceBasedTuner}. - */ - public static ResourceBasedSlotSupplier createForWorkflow( - ResourceBasedController resourceBasedController, ResourceBasedSlotOptions options) { - return new ResourceBasedSlotSupplier<>( - WorkflowSlotInfo.class, resourceBasedController, options); - } - - /** - * Construct a slot supplier for activity tasks with the given resource controller and options. - * - *

The resource controller must be the same among all slot suppliers in a worker. If you want - * to use resource-based tuning for all slot suppliers, prefer {@link ResourceBasedTuner}. - */ - public static ResourceBasedSlotSupplier createForActivity( - ResourceBasedController resourceBasedController, ResourceBasedSlotOptions options) { - return new ResourceBasedSlotSupplier<>( - ActivitySlotInfo.class, resourceBasedController, options); - } - - /** - * Construct a slot supplier for local activities with the given resource controller and options. - * - *

The resource controller must be the same among all slot suppliers in a worker. If you want - * to use resource-based tuning for all slot suppliers, prefer {@link ResourceBasedTuner}. - */ - public static ResourceBasedSlotSupplier createForLocalActivity( - ResourceBasedController resourceBasedController, ResourceBasedSlotOptions options) { - return new ResourceBasedSlotSupplier<>( - LocalActivitySlotInfo.class, resourceBasedController, options); - } - - private ResourceBasedSlotSupplier( - Class clazz, - ResourceBasedController resourceBasedController, - ResourceBasedSlotOptions options) { - this.resourceController = resourceBasedController; - // Merge default options for any unset fields - if (WorkflowSlotInfo.class.isAssignableFrom(clazz)) { - this.options = - ResourceBasedSlotOptions.newBuilder() - .setMinimumSlots( - options.getMinimumSlots() == 0 - ? ResourceBasedTuner.DEFAULT_WORKFLOW_SLOT_OPTIONS.getMinimumSlots() - : options.getMinimumSlots()) - .setMaximumSlots( - options.getMaximumSlots() == 0 - ? ResourceBasedTuner.DEFAULT_WORKFLOW_SLOT_OPTIONS.getMaximumSlots() - : options.getMaximumSlots()) - .setRampThrottle( - options.getRampThrottle() == null - ? ResourceBasedTuner.DEFAULT_WORKFLOW_SLOT_OPTIONS.getRampThrottle() - : options.getRampThrottle()) - .build(); - } else { - this.options = - ResourceBasedSlotOptions.newBuilder() - .setMinimumSlots( - options.getMinimumSlots() == 0 - ? ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS.getMinimumSlots() - : options.getMinimumSlots()) - .setMaximumSlots( - options.getMaximumSlots() == 0 - ? ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS.getMaximumSlots() - : options.getMaximumSlots()) - .setRampThrottle( - options.getRampThrottle() == null - ? ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS.getRampThrottle() - : options.getRampThrottle()) - .build(); - } - } - - @Override - public SlotPermit reserveSlot(SlotReserveContext ctx) throws InterruptedException { - while (true) { - if (ctx.getNumIssuedSlots() < options.getMinimumSlots()) { - return new SlotPermit(); - } else { - Duration mustWaitFor; - try { - mustWaitFor = options.getRampThrottle().minus(timeSinceLastSlotIssued()); - } catch (ArithmeticException e) { - mustWaitFor = Duration.ZERO; - } - if (mustWaitFor.compareTo(Duration.ZERO) > 0) { - Thread.sleep(mustWaitFor.toMillis()); - } - - Optional permit = tryReserveSlot(ctx); - if (permit.isPresent()) { - return permit.get(); - } else { - Thread.sleep(10); - } - } - } - } - - @Override - public Optional tryReserveSlot(SlotReserveContext ctx) { - int numIssued = ctx.getNumIssuedSlots(); - if (numIssued < options.getMinimumSlots() - || (timeSinceLastSlotIssued().compareTo(options.getRampThrottle()) > 0 - && numIssued < options.getMaximumSlots() - && resourceController.pidDecision())) { - lastSlotIssuedAt = Instant.now(); - return Optional.of(new SlotPermit()); - } - return Optional.empty(); - } - - @Override - public void markSlotUsed(SlotMarkUsedContext ctx) {} - - @Override - public void releaseSlot(SlotReleaseContext ctx) {} - - public ResourceBasedController getResourceController() { - return resourceController; - } - - private Duration timeSinceLastSlotIssued() { - return Duration.between(lastSlotIssuedAt, Instant.now()); - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedTuner.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedTuner.java deleted file mode 100644 index 47ad34e19..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/ResourceBasedTuner.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.time.Duration; -import javax.annotation.Nonnull; - -/** A {@link WorkerTuner} that attempts to allocate slots based on available system resources. */ -@Experimental -public class ResourceBasedTuner implements WorkerTuner { - public static final ResourceBasedSlotOptions DEFAULT_WORKFLOW_SLOT_OPTIONS = - ResourceBasedSlotOptions.newBuilder() - .setMinimumSlots(5) - .setMaximumSlots(500) - .setRampThrottle(Duration.ZERO) - .build(); - public static final ResourceBasedSlotOptions DEFAULT_ACTIVITY_SLOT_OPTIONS = - ResourceBasedSlotOptions.newBuilder() - .setMinimumSlots(1) - .setMaximumSlots(1000) - .setRampThrottle(Duration.ofMillis(50)) - .build(); - - private final ResourceBasedController controller; - private final ResourceBasedSlotOptions workflowSlotOptions; - private final ResourceBasedSlotOptions activitySlotOptions; - private final ResourceBasedSlotOptions localActivitySlotOptions; - - public static Builder newBuilder() { - return new Builder(); - } - - public static final class Builder { - private ResourceBasedControllerOptions controllerOptions; - private @Nonnull ResourceBasedSlotOptions workflowSlotOptions = DEFAULT_WORKFLOW_SLOT_OPTIONS; - private @Nonnull ResourceBasedSlotOptions activitySlotOptions = DEFAULT_ACTIVITY_SLOT_OPTIONS; - private @Nonnull ResourceBasedSlotOptions localActivitySlotOptions = - DEFAULT_ACTIVITY_SLOT_OPTIONS; - - private Builder() {} - - public Builder setControllerOptions(ResourceBasedControllerOptions controllerOptions) { - this.controllerOptions = controllerOptions; - return this; - } - - /** - * Set the slot options for workflow tasks. Has no effect after the worker using this tuner - * starts. - * - *

Defaults to minimum 5 slots, maximum 500 slots, and no ramp throttle. - */ - public Builder setWorkflowSlotOptions(@Nonnull ResourceBasedSlotOptions workflowSlotOptions) { - this.workflowSlotOptions = workflowSlotOptions; - return this; - } - - /** - * Set the slot options for activity tasks. Has no effect after the worker using this tuner - * starts. - * - *

Defaults to minimum 1 slot, maximum 1000 slots, and 50ms ramp throttle. - */ - public Builder setActivitySlotOptions(@Nonnull ResourceBasedSlotOptions activitySlotOptions) { - this.activitySlotOptions = activitySlotOptions; - return this; - } - - /** - * Set the slot options for local activity tasks. Has no effect after the worker using this - * tuner starts. - * - *

Defaults to minimum 1 slot, maximum 1000 slots, and 50ms ramp throttle. - */ - public Builder setLocalActivitySlotOptions( - @Nonnull ResourceBasedSlotOptions localActivitySlotOptions) { - this.localActivitySlotOptions = localActivitySlotOptions; - return this; - } - - public ResourceBasedTuner build() { - return new ResourceBasedTuner( - controllerOptions, workflowSlotOptions, activitySlotOptions, localActivitySlotOptions); - } - } - - /** - * @param controllerOptions options for the {@link ResourceBasedController} used by this tuner - */ - public ResourceBasedTuner( - ResourceBasedControllerOptions controllerOptions, - ResourceBasedSlotOptions workflowSlotOptions, - ResourceBasedSlotOptions activitySlotOptions, - ResourceBasedSlotOptions localActivitySlotOptions) { - this.controller = ResourceBasedController.newSystemInfoController(controllerOptions); - this.workflowSlotOptions = workflowSlotOptions; - this.activitySlotOptions = activitySlotOptions; - this.localActivitySlotOptions = localActivitySlotOptions; - } - - @Nonnull - @Override - public SlotSupplier getWorkflowTaskSlotSupplier() { - return ResourceBasedSlotSupplier.createForWorkflow(controller, workflowSlotOptions); - } - - @Nonnull - @Override - public SlotSupplier getActivityTaskSlotSupplier() { - return ResourceBasedSlotSupplier.createForActivity(controller, activitySlotOptions); - } - - @Nonnull - @Override - public SlotSupplier getLocalActivitySlotSupplier() { - return ResourceBasedSlotSupplier.createForLocalActivity(controller, localActivitySlotOptions); - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotInfo.java deleted file mode 100644 index 16c4d17fe..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotInfo.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; - -/** The base class that all slot info types used by {@link SlotSupplier} extend. */ -@Experimental -public abstract class SlotInfo { - SlotInfo() {} -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotMarkUsedContext.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotMarkUsedContext.java deleted file mode 100644 index 24ecba9d1..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotMarkUsedContext.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; - -@Experimental -public interface SlotMarkUsedContext { - /** - * @return The information associated with the slot that is being marked as used. - */ - SI getSlotInfo(); - - /** - * @return The previously reserved permit that is being used with this slot. - */ - SlotPermit getSlotPermit(); -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotPermit.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotPermit.java deleted file mode 100644 index 4fd625cba..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotPermit.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; - -/** - * This class is handed out by implementations of {@link SlotSupplier}. Permits are held until the - * tasks they are associated with (if any) are finished processing, or if the reservation is no - * longer needed. Your supplier implementation may store additional data in the permit, if desired. - * - *

When {@link SlotSupplier#releaseSlot(SlotReleaseContext)} is called, the exact same instance - * of the permit is passed back to the supplier. - */ -@Experimental -public final class SlotPermit { - public final Object userData; - - public SlotPermit() { - this.userData = null; - } - - public SlotPermit(Object userData) { - this.userData = userData; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseContext.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseContext.java deleted file mode 100644 index bf9600e99..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import javax.annotation.Nullable; - -@Experimental -public interface SlotReleaseContext { - /** - * @return The reason the slot is being released. - */ - SlotReleaseReason getSlotReleaseReason(); - - /** - * @return The permit the slot was using that is now being released. - */ - SlotPermit getSlotPermit(); - - /** - * @return The information associated with the slot that is being released. May be null if the - * slot was never marked as used. - */ - @Nullable - SI getSlotInfo(); -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseReason.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseReason.java deleted file mode 100644 index 438a7a907..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReleaseReason.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import javax.annotation.Nullable; - -@Experimental -public abstract class SlotReleaseReason { - SlotReleaseReason() {} - - public static SlotReleaseReason taskComplete() { - return new TaskComplete(); - } - - public static SlotReleaseReason neverUsed() { - return new NeverUsed(); - } - - public static SlotReleaseReason error(Exception exception) { - return new Error(exception); - } - - public boolean isError() { - return false; - } - - /** - * @return the exception that caused the slot to be released, if this is a reason of type {@link - * Error}. - */ - public @Nullable Exception getException() { - return null; - } - - /** The slot was released because the task was completed (regardless of status). */ - public static class TaskComplete extends SlotReleaseReason {} - - /** The slot was released because it was never needed. */ - public static class NeverUsed extends SlotReleaseReason {} - - /** - * The slot was released because some error was encountered before the slot could be used to - * actually process the task. - */ - public static class Error extends SlotReleaseReason { - private final Exception exception; - - private Error(Exception exception) { - this.exception = exception; - } - - @Override - public boolean isError() { - return true; - } - - @Override - public Exception getException() { - return exception; - } - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReserveContext.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReserveContext.java deleted file mode 100644 index 57007d57d..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotReserveContext.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.util.Map; - -@Experimental -public interface SlotReserveContext { - /** - * @return the Task Queue for which this reservation request is associated. - */ - String getTaskQueue(); - - /** - * @return A read-only & safe for concurrent access mapping of slot permits to the information - * associated with the in-use slot. This map is changed internally any time new slots are - * used. - */ - Map getUsedSlots(); - - /** - * @return The worker's identity that is associated with this reservation request. - */ - String getWorkerIdentity(); - - /** - * @return The worker's build ID that is associated with this reservation request. - */ - String getWorkerBuildId(); - - /** - * @return The number of currently outstanding slot permits of this type, whether used or not. - */ - int getNumIssuedSlots(); -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotSupplier.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotSupplier.java deleted file mode 100644 index 27a74dc9f..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SlotSupplier.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import java.util.Optional; - -/** - * A SlotSupplier is responsible for managing the number of slots available for a given type of - * task. The three types of tasks are workflow, activity, and local activity. Implementing this - * interface allows you to carefully control how many tasks of any given type a worker will process - * at once. - * - * @param The type of information that will be used to reserve a slot. The three info types are - * {@link WorkflowSlotInfo}, {@link ActivitySlotInfo}, and {@link LocalActivitySlotInfo}. - */ -@Experimental -public interface SlotSupplier { - /** - * This function is called before polling for new tasks. Your implementation should block until a - * slot is available, then return a permit to use that slot. - * - * @param ctx The context for slot reservation. - * @return A permit to use the slot which may be populated with your own data. - * @throws InterruptedException The worker may choose to interrupt the thread in order to cancel - * the reservation, or during shutdown. You may perform cleanup, and then should rethrow the - * exception. - */ - SlotPermit reserveSlot(SlotReserveContext ctx) throws InterruptedException; - - /** - * This function is called when trying to reserve slots for "eager" workflow and activity tasks. - * Eager tasks are those which are returned as a result of completing a workflow task, rather than - * from polling. Your implementation must not block, and If a slot is available, return a permit - * to use that slot. - * - * @param ctx The context for slot reservation. - * @return Maybe a permit to use the slot which may be populated with your own data. - */ - Optional tryReserveSlot(SlotReserveContext ctx); - - /** - * This function is called once a slot is actually being used to process some task, which may be - * some time after the slot was reserved originally. For example, if there is no work for a - * worker, a number of slots equal to the number of active pollers may already be reserved, but - * none of them are being used yet. This call should be non-blocking. - * - * @param ctx The context for marking a slot as used. - */ - void markSlotUsed(SlotMarkUsedContext ctx); - - /** - * This function is called once a permit is no longer needed. This could be because the task has - * finished, whether successfully or not, or because the slot was no longer needed (ex: the number - * of active pollers decreased). This call should be non-blocking. - * - * @param ctx The context for releasing a slot. - */ - void releaseSlot(SlotReleaseContext ctx); - - /** - * Because we currently use thread pools to execute tasks, there must be *some* defined - * upper-limit on the size of the thread pool for each kind of task. You must not hand out more - * permits than this number. If unspecified, the default is {@link Integer#MAX_VALUE}. Be aware - * that if your implementation hands out unreasonable numbers of permits, you could easily - * oversubscribe the worker, and cause it to run out of resources. - * - *

This value should never change during the lifetime of the supplier. - * - * @return the maximum number of slots that can ever be in use at one type for this slot type. - */ - default int getMaximumSlots() { - return Integer.MAX_VALUE; - } -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SystemResourceInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/SystemResourceInfo.java deleted file mode 100644 index ff5cf6a07..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/SystemResourceInfo.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; - -/** Implementors determine how resource usage is measured. */ -@Experimental -public interface SystemResourceInfo { - /** - * @return System-wide CPU usage as a percentage [0.0, 1.0] - */ - double getCPUUsagePercent(); - - /** - * @return Memory usage as a percentage [0.0, 1.0]. Memory usage should reflect either system-wide - * usage or JVM-specific usage, whichever is higher, to avoid running out of memory in either - * way. - */ - double getMemoryUsagePercent(); -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkerTuner.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkerTuner.java deleted file mode 100644 index a25099569..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkerTuner.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.common.Experimental; -import javax.annotation.Nonnull; - -/** WorkerTuners allow for the dynamic customization of some aspects of worker configuration. */ -@Experimental -public interface WorkerTuner { - /** - * @return A {@link SlotSupplier} for workflow tasks. - */ - @Nonnull - SlotSupplier getWorkflowTaskSlotSupplier(); - - /** - * @return A {@link SlotSupplier} for activity tasks. - */ - @Nonnull - SlotSupplier getActivityTaskSlotSupplier(); - - /** - * @return A {@link SlotSupplier} for local activities. - */ - @Nonnull - SlotSupplier getLocalActivitySlotSupplier(); -} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkflowSlotInfo.java b/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkflowSlotInfo.java deleted file mode 100644 index 0995603a9..000000000 --- a/temporal-sdk/src/main/java/io/temporal/worker/tuning/WorkflowSlotInfo.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker.tuning; - -import io.temporal.api.enums.v1.TaskQueueKind; -import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueRequest; -import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse; -import io.temporal.common.Experimental; -import java.util.Objects; -import javax.annotation.Nonnull; - -/** Contains information about a slot that is being used to execute a workflow task. */ -@Experimental -public class WorkflowSlotInfo extends SlotInfo { - private final String workflowType; - private final String taskQueue; - private final String workflowId; - private final String runId; - private final String workerIdentity; - private final String workerBuildId; - private final boolean fromStickyQueue; - - /** Don't rely on this constructor. It is for internal use by the SDK. */ - public WorkflowSlotInfo( - @Nonnull PollWorkflowTaskQueueResponse response, - @Nonnull PollWorkflowTaskQueueRequest request) { - this.workflowType = response.getWorkflowType().getName(); - this.taskQueue = request.getTaskQueue().getNormalName(); - this.workflowId = response.getWorkflowExecution().getWorkflowId(); - this.runId = response.getWorkflowExecution().getRunId(); - this.workerIdentity = request.getIdentity(); - this.workerBuildId = request.getWorkerVersionCapabilities().getBuildId(); - this.fromStickyQueue = request.getTaskQueue().getKind() == TaskQueueKind.TASK_QUEUE_KIND_STICKY; - } - - /** Don't rely on this constructor. It is for internal use by the SDK. */ - public WorkflowSlotInfo( - String workflowType, - String taskQueue, - String workflowId, - String runId, - String workerIdentity, - String workerBuildId, - boolean fromStickyQueue) { - this.workflowType = workflowType; - this.taskQueue = taskQueue; - this.workflowId = workflowId; - this.runId = runId; - this.workerIdentity = workerIdentity; - this.workerBuildId = workerBuildId; - this.fromStickyQueue = fromStickyQueue; - } - - public String getWorkflowType() { - return workflowType; - } - - public String getWorkflowId() { - return workflowId; - } - - public String getRunId() { - return runId; - } - - public String getTaskQueue() { - return taskQueue; - } - - public String getWorkerIdentity() { - return workerIdentity; - } - - public String getWorkerBuildId() { - return workerBuildId; - } - - public boolean isFromStickyQueue() { - return fromStickyQueue; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - WorkflowSlotInfo that = (WorkflowSlotInfo) o; - return fromStickyQueue == that.fromStickyQueue - && Objects.equals(workflowType, that.workflowType) - && Objects.equals(taskQueue, that.taskQueue) - && Objects.equals(workflowId, that.workflowId) - && Objects.equals(runId, that.runId) - && Objects.equals(workerIdentity, that.workerIdentity) - && Objects.equals(workerBuildId, that.workerBuildId); - } - - @Override - public int hashCode() { - return Objects.hash( - workflowType, taskQueue, workflowId, runId, workerIdentity, workerBuildId, fromStickyQueue); - } - - @Override - public String toString() { - return "WorkflowSlotInfo{" - + "workflowType='" - + workflowType - + '\'' - + ", taskQueue='" - + taskQueue - + '\'' - + ", workflowId='" - + workflowId - + '\'' - + ", runId='" - + runId - + '\'' - + ", workerIdentity='" - + workerIdentity - + '\'' - + ", workerBuildId='" - + workerBuildId - + '\'' - + ", fromStickyQueue=" - + fromStickyQueue - + '}'; - } -} diff --git a/temporal-sdk/src/test/java/io/temporal/common/reporter/TestStatsReporter.java b/temporal-sdk/src/test/java/io/temporal/common/reporter/TestStatsReporter.java index 506a5a299..661cd0660 100644 --- a/temporal-sdk/src/test/java/io/temporal/common/reporter/TestStatsReporter.java +++ b/temporal-sdk/src/test/java/io/temporal/common/reporter/TestStatsReporter.java @@ -193,6 +193,7 @@ private String getMetricName(String name, Map tags) { return name + " " + tags.entrySet().stream() + .filter(e -> e.getValue() != "none") .map(Map.Entry::toString) .sorted() .collect(joining("|", "[", "]")); diff --git a/temporal-sdk/src/test/java/io/temporal/internal/worker/SlotSupplierTest.java b/temporal-sdk/src/test/java/io/temporal/internal/worker/SlotSupplierTest.java deleted file mode 100644 index e14e0bca0..000000000 --- a/temporal-sdk/src/test/java/io/temporal/internal/worker/SlotSupplierTest.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.internal.worker; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.Assert.*; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.*; - -import com.google.protobuf.ByteString; -import com.uber.m3.tally.RootScopeBuilder; -import com.uber.m3.tally.Scope; -import io.temporal.api.common.v1.WorkflowExecution; -import io.temporal.api.common.v1.WorkflowType; -import io.temporal.api.workflowservice.v1.GetSystemInfoResponse; -import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse; -import io.temporal.api.workflowservice.v1.WorkflowServiceGrpc; -import io.temporal.common.reporter.TestStatsReporter; -import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.worker.tuning.*; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.mockito.stubbing.OngoingStubbing; - -@RunWith(Parameterized.class) -public class SlotSupplierTest { - private final TestStatsReporter reporter = new TestStatsReporter(); - private static final String WORKFLOW_ID = "test-workflow-id"; - private static final String RUN_ID = "test-run-id"; - private static final String WORKFLOW_TYPE = "test-workflow-type"; - private static final String TASK_QUEUE = "test-task-queue"; - - @Parameterized.Parameter public boolean throwOnPoll; - - @Parameterized.Parameters() - public static Object[] data() { - return new Object[][] {{true}, {false}}; - } - - @Test - public void supplierIsCalledAppropriately() throws InterruptedException { - WorkflowServiceStubs client = mock(WorkflowServiceStubs.class); - when(client.getServerCapabilities()) - .thenReturn(() -> GetSystemInfoResponse.Capabilities.newBuilder().build()); - WorkflowServiceGrpc.WorkflowServiceBlockingStub blockingStub = - mock(WorkflowServiceGrpc.WorkflowServiceBlockingStub.class); - when(client.blockingStub()).thenReturn(blockingStub); - when(blockingStub.withOption(any(), any())).thenReturn(blockingStub); - - SlotSupplier mockSupplier = mock(SlotSupplier.class); - AtomicInteger usedSlotsWhenCalled = new AtomicInteger(-1); - when(mockSupplier.reserveSlot( - argThat( - src -> { - usedSlotsWhenCalled.set(src.getUsedSlots().size()); - return true; - }))) - .thenReturn(new SlotPermit()); - - TrackingSlotSupplier trackingSS = new TrackingSlotSupplier<>(mockSupplier); - StickyQueueBalancer stickyQueueBalancer = new StickyQueueBalancer(5, true); - - Scope metricsScope = - new RootScopeBuilder() - .reporter(reporter) - .reportEvery(com.uber.m3.util.Duration.ofMillis(1)); - WorkflowPollTask poller = - new WorkflowPollTask( - client, - "default", - TASK_QUEUE, - "stickytaskqueue", - "", - "", - false, - trackingSS, - stickyQueueBalancer, - metricsScope, - () -> GetSystemInfoResponse.Capabilities.newBuilder().build()); - - PollWorkflowTaskQueueResponse pollResponse = - PollWorkflowTaskQueueResponse.newBuilder() - .setTaskToken(ByteString.copyFrom("token", UTF_8)) - .setWorkflowExecution( - WorkflowExecution.newBuilder().setWorkflowId(WORKFLOW_ID).setRunId(RUN_ID).build()) - .setWorkflowType(WorkflowType.newBuilder().setName(WORKFLOW_TYPE).build()) - .build(); - - OngoingStubbing pollMock = - when(blockingStub.pollWorkflowTaskQueue(any())); - if (throwOnPoll) { - pollMock.thenThrow(new RuntimeException("Poll failed")); - } else { - pollMock.thenReturn(pollResponse); - } - - if (throwOnPoll) { - assertThrows(RuntimeException.class, () -> poller.poll()); - verify(mockSupplier, times(1)).reserveSlot(any()); - verify(mockSupplier, times(1)).releaseSlot(any()); - assertEquals(0, trackingSS.getUsedSlots().size()); - } else { - WorkflowTask task = poller.poll(); - assertNotNull(task); - // We can't test this in the verifier, since it will get an up-to-date reference to the map - // where the slot *is* used. - assertEquals(0, usedSlotsWhenCalled.get()); - verify(mockSupplier, times(1)) - .reserveSlot(argThat(arg -> Objects.equals(arg.getTaskQueue(), TASK_QUEUE))); - verify(mockSupplier, times(0)).releaseSlot(any()); - assertEquals(1, trackingSS.getUsedSlots().size()); - } - } -} diff --git a/temporal-sdk/src/test/java/io/temporal/internal/worker/StickyQueueBacklogTest.java b/temporal-sdk/src/test/java/io/temporal/internal/worker/StickyQueueBacklogTest.java index 0df7e783d..67a325d40 100644 --- a/temporal-sdk/src/test/java/io/temporal/internal/worker/StickyQueueBacklogTest.java +++ b/temporal-sdk/src/test/java/io/temporal/internal/worker/StickyQueueBacklogTest.java @@ -41,8 +41,7 @@ import io.temporal.api.workflowservice.v1.WorkflowServiceGrpc; import io.temporal.common.reporter.TestStatsReporter; import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.worker.tuning.FixedSizeSlotSupplier; -import io.temporal.worker.tuning.WorkflowSlotInfo; +import java.util.concurrent.Semaphore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -73,8 +72,7 @@ public void stickyQueueBacklogResetTest() { when(client.blockingStub()).thenReturn(blockingStub); when(blockingStub.withOption(any(), any())).thenReturn(blockingStub); - TrackingSlotSupplier slotSupplier = - new TrackingSlotSupplier<>(new FixedSizeSlotSupplier<>(10)); + Semaphore executorSlotsSemaphore = new Semaphore(10); StickyQueueBalancer stickyQueueBalancer = new StickyQueueBalancer(2, true); Scope metricsScope = @@ -90,7 +88,7 @@ public void stickyQueueBacklogResetTest() { "", "", false, - slotSupplier, + executorSlotsSemaphore, stickyQueueBalancer, metricsScope, () -> GetSystemInfoResponse.Capabilities.newBuilder().build()); diff --git a/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowSlotTests.java b/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowSlotTests.java new file mode 100644 index 000000000..fdab84da9 --- /dev/null +++ b/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowSlotTests.java @@ -0,0 +1,253 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material 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 io.temporal.internal.worker; + +import com.uber.m3.tally.RootScopeBuilder; +import com.uber.m3.tally.Scope; +import com.uber.m3.util.ImmutableMap; +import io.temporal.activity.ActivityInterface; +import io.temporal.activity.ActivityMethod; +import io.temporal.activity.ActivityOptions; +import io.temporal.activity.LocalActivityOptions; +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowOptions; +import io.temporal.common.RetryOptions; +import io.temporal.common.reporter.TestStatsReporter; +import io.temporal.testing.internal.SDKTestWorkflowRule; +import io.temporal.worker.MetricsType; +import io.temporal.worker.WorkerOptions; +import io.temporal.workflow.SignalMethod; +import io.temporal.workflow.Workflow; +import io.temporal.workflow.WorkflowInterface; +import io.temporal.workflow.WorkflowMethod; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +public class WorkflowSlotTests { + private final int MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE = 100; + private final int MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE = 1000; + private final int MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE = 10000; + private final TestStatsReporter reporter = new TestStatsReporter(); + static CountDownLatch activityBlockLatch = new CountDownLatch(1); + static CountDownLatch activityRunningLatch = new CountDownLatch(1); + + Scope metricsScope = + new RootScopeBuilder().reporter(reporter).reportEvery(com.uber.m3.util.Duration.ofMillis(1)); + + @Rule + public SDKTestWorkflowRule testWorkflowRule = + SDKTestWorkflowRule.newBuilder() + .setWorkerOptions( + WorkerOptions.newBuilder() + .setMaxConcurrentWorkflowTaskExecutionSize( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE) + .setMaxConcurrentActivityExecutionSize(MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE) + .setMaxConcurrentLocalActivityExecutionSize( + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE) + .build()) + .setMetricsScope(metricsScope) + .setActivityImplementations(new TestActivityImpl()) + .setWorkflowTypes(SleepingWorkflowImpl.class) + .setDoNotStart(true) + .build(); + + @Before + public void setup() { + reporter.flush(); + activityBlockLatch = new CountDownLatch(1); + activityRunningLatch = new CountDownLatch(1); + } + + private void assertWorkerSlotCount(int worker, int activity, int localActivity) { + try { + // There can be a delay in metrics emission, another option if this + // is too flaky is to poll the metrics. + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + reporter.assertGauge( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, getWorkerTags("WorkflowWorker"), worker); + // All slots should be available + reporter.assertGauge( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, getWorkerTags("ActivityWorker"), activity); + // All slots should be available + reporter.assertGauge( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, + getWorkerTags("LocalActivityWorker"), + localActivity); + } + + @WorkflowInterface + public interface TestWorkflow { + @WorkflowMethod + String workflow(boolean useLocalActivity); + + @SignalMethod + void unblock(); + } + + public static class SleepingWorkflowImpl implements TestWorkflow { + boolean unblocked = false; + + private final TestActivity activity = + Workflow.newActivityStub( + TestActivity.class, + ActivityOptions.newBuilder() + .setStartToCloseTimeout(Duration.ofSeconds(10)) + .setRetryOptions(RetryOptions.newBuilder().setMaximumAttempts(1).build()) + .validateAndBuildWithDefaults()); + + private final TestActivity localActivity = + Workflow.newLocalActivityStub( + TestActivity.class, + LocalActivityOptions.newBuilder() + .setStartToCloseTimeout(Duration.ofSeconds(10)) + .setRetryOptions(RetryOptions.newBuilder().setMaximumAttempts(1).build()) + .validateAndBuildWithDefaults()); + + @Override + public String workflow(boolean useLocalActivity) { + Workflow.await(() -> unblocked); + if (useLocalActivity) { + localActivity.activity("test"); + } else { + activity.activity("test"); + } + return "ok"; + } + + @Override + public void unblock() { + unblocked = true; + } + } + + @ActivityInterface + public interface TestActivity { + + @ActivityMethod + String activity(String input); + } + + public static class TestActivityImpl implements TestActivity { + @Override + public String activity(String input) { + activityRunningLatch.countDown(); + try { + activityBlockLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return ""; + } + } + + private Map getWorkerTags(String workerType) { + return ImmutableMap.of( + "worker_type", + workerType, + "task_queue", + testWorkflowRule.getTaskQueue(), + "namespace", + "UnitTest"); + } + + @Test + public void TestTaskSlotsEmittedOnStart() { + // Verify that no metrics are emitted before the worker is started + reporter.assertNoMetric( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, getWorkerTags("WorkflowWorker")); + reporter.assertNoMetric( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, getWorkerTags("ActivityWorker")); + reporter.assertNoMetric( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, getWorkerTags("LocalActivityWorker")); + // Start the worker + testWorkflowRule.getTestEnvironment().start(); + // All slots should be available + assertWorkerSlotCount( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE, + MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE, + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE); + } + + @Test + public void TestActivityTaskSlots() throws InterruptedException { + testWorkflowRule.getTestEnvironment().start(); + WorkflowClient client = testWorkflowRule.getWorkflowClient(); + TestWorkflow workflow = + client.newWorkflowStub( + TestWorkflow.class, + WorkflowOptions.newBuilder() + .setTaskQueue(testWorkflowRule.getTaskQueue()) + .validateBuildWithDefaults()); + WorkflowClient.start(workflow::workflow, false); + workflow.unblock(); + activityRunningLatch.await(); + // The activity slot should be taken and the workflow slot should not be taken + assertWorkerSlotCount( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE, + MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE - 1, + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE); + + activityBlockLatch.countDown(); + // Wait for the workflow to finish + workflow.workflow(false); + // All slots should be available + assertWorkerSlotCount( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE, + MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE, + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE); + } + + @Test + public void TestLocalActivityTaskSlots() throws InterruptedException { + testWorkflowRule.getTestEnvironment().start(); + WorkflowClient client = testWorkflowRule.getWorkflowClient(); + TestWorkflow workflow = + client.newWorkflowStub( + TestWorkflow.class, + WorkflowOptions.newBuilder() + .setTaskQueue(testWorkflowRule.getTaskQueue()) + .validateBuildWithDefaults()); + WorkflowClient.start(workflow::workflow, true); + workflow.unblock(); + activityRunningLatch.await(); + // The local activity slot should be taken and the workflow slot should be taken + assertWorkerSlotCount( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE - 1, + MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE, + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE - 1); + + activityBlockLatch.countDown(); + // Wait for the workflow to finish + workflow.workflow(true); + // All slots should be available + assertWorkerSlotCount( + MAX_CONCURRENT_WORKFLOW_TASK_EXECUTION_SIZE, + MAX_CONCURRENT_ACTIVITY_EXECUTION_SIZE, + MAX_CONCURRENT_LOCAL_ACTIVITY_EXECUTION_SIZE); + } +} diff --git a/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowWorkerTest.java b/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowWorkerTest.java index 6c1bd359e..a8645f209 100644 --- a/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowWorkerTest.java +++ b/temporal-sdk/src/test/java/io/temporal/internal/worker/WorkflowWorkerTest.java @@ -42,8 +42,6 @@ import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.testUtils.HistoryUtils; import io.temporal.worker.MetricsType; -import io.temporal.worker.tuning.FixedSizeSlotSupplier; -import io.temporal.worker.tuning.WorkflowSlotInfo; import java.time.Duration; import java.util.UUID; import java.util.concurrent.*; @@ -73,8 +71,6 @@ public void concurrentPollRequestLockTest() throws Exception { new RootScopeBuilder() .reporter(reporter) .reportEvery(com.uber.m3.util.Duration.ofMillis(1)); - TrackingSlotSupplier slotSupplier = - new TrackingSlotSupplier<>(new FixedSizeSlotSupplier<>(100)); WorkflowExecutorCache cache = new WorkflowExecutorCache(10, runLockManager, metricsScope); WorkflowTaskHandler taskHandler = mock(WorkflowTaskHandler.class); @@ -96,8 +92,7 @@ public void concurrentPollRequestLockTest() throws Exception { runLockManager, cache, taskHandler, - eagerActivityDispatcher, - slotSupplier); + eagerActivityDispatcher); WorkflowServiceGrpc.WorkflowServiceBlockingStub blockingStub = mock(WorkflowServiceGrpc.WorkflowServiceBlockingStub.class); @@ -175,6 +170,11 @@ public void concurrentPollRequestLockTest() throws Exception { }); assertTrue(worker.start()); + // All slots should be available + reporter.assertGauge( + MetricsType.WORKER_TASK_SLOTS_AVAILABLE, + ImmutableMap.of("worker_type", "WorkflowWorker"), + 100.0); // Wait until we have got all the polls pollTaskQueueLatch.await(); // Wait until the worker handles at least one WFT @@ -221,8 +221,6 @@ public void respondWorkflowTaskFailureMetricTest() throws Exception { .reporter(reporter) .reportEvery(com.uber.m3.util.Duration.ofMillis(1)); WorkflowExecutorCache cache = new WorkflowExecutorCache(10, runLockManager, metricsScope); - TrackingSlotSupplier slotSupplier = - new TrackingSlotSupplier<>(new FixedSizeSlotSupplier<>(10)); WorkflowTaskHandler taskHandler = mock(WorkflowTaskHandler.class); when(taskHandler.isAnyTypeSupported()).thenReturn(true); @@ -243,8 +241,7 @@ public void respondWorkflowTaskFailureMetricTest() throws Exception { runLockManager, cache, taskHandler, - eagerActivityDispatcher, - slotSupplier); + eagerActivityDispatcher); WorkflowServiceGrpc.WorkflowServiceBlockingStub blockingStub = mock(WorkflowServiceGrpc.WorkflowServiceBlockingStub.class); @@ -318,9 +315,6 @@ public void resetWorkflowIdFromWorkflowTaskTest() throws Throwable { Scope metricScope = new NoopScope(); WorkflowExecutorCache cache = new WorkflowExecutorCache(1, runLockManager, metricScope); - TrackingSlotSupplier slotSupplier = - new TrackingSlotSupplier<>(new FixedSizeSlotSupplier<>(1)); - WorkflowTaskHandler rootTaskHandler = new ReplayWorkflowTaskHandler( "namespace", @@ -375,8 +369,7 @@ public boolean isAnyTypeSupported() { runLockManager, cache, taskHandler, - eagerActivityDispatcher, - slotSupplier); + eagerActivityDispatcher); WorkflowServiceGrpc.WorkflowServiceBlockingStub blockingStub = mock(WorkflowServiceGrpc.WorkflowServiceBlockingStub.class); diff --git a/temporal-sdk/src/test/java/io/temporal/worker/IndependentResourceBasedTests.java b/temporal-sdk/src/test/java/io/temporal/worker/IndependentResourceBasedTests.java deleted file mode 100644 index c78f943a1..000000000 --- a/temporal-sdk/src/test/java/io/temporal/worker/IndependentResourceBasedTests.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker; - -interface IndependentResourceBasedTests {} diff --git a/temporal-sdk/src/test/java/io/temporal/worker/ResourceBasedTunerTests.java b/temporal-sdk/src/test/java/io/temporal/worker/ResourceBasedTunerTests.java deleted file mode 100644 index 58974e28a..000000000 --- a/temporal-sdk/src/test/java/io/temporal/worker/ResourceBasedTunerTests.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. - * - * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. - * - * Modifications copyright (C) 2017 Uber Technologies, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this material 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 io.temporal.worker; - -import static io.temporal.testing.internal.SDKTestWorkflowRule.NAMESPACE; - -import com.uber.m3.tally.RootScopeBuilder; -import com.uber.m3.util.ImmutableMap; -import io.temporal.activity.ActivityInterface; -import io.temporal.activity.ActivityOptions; -import io.temporal.common.reporter.TestStatsReporter; -import io.temporal.serviceclient.MetricsTag; -import io.temporal.testing.internal.SDKTestWorkflowRule; -import io.temporal.worker.tuning.*; -import io.temporal.workflow.*; -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -public class ResourceBasedTunerTests { - - private final TestStatsReporter reporter = new TestStatsReporter(); - private static final Map TAGS_NAMESPACE = - new ImmutableMap.Builder().putAll(MetricsTag.defaultTags(NAMESPACE)).build(); - - @Rule - public SDKTestWorkflowRule testWorkflowRule = - SDKTestWorkflowRule.newBuilder() - .setWorkerOptions( - WorkerOptions.newBuilder() - .setWorkerTuner( - ResourceBasedTuner.newBuilder() - .setControllerOptions( - ResourceBasedControllerOptions.newBuilder(0.7, 0.7).build()) - .build()) - .build()) - .setActivityImplementations(new ActivitiesImpl()) - .setWorkflowTypes(ResourceTunerWorkflowImpl.class) - .setMetricsScope( - new RootScopeBuilder() - .reporter(reporter) - .reportEvery(com.uber.m3.util.Duration.ofMillis(10))) - .build(); - - @Test - public void canRunWithResourceBasedTuner() { - ResourceTunerWorkflow workflow = testWorkflowRule.newWorkflowStub(ResourceTunerWorkflow.class); - workflow.execute(10, 1000); - Map nsAndTaskQueue = - new ImmutableMap.Builder() - .putAll(TAGS_NAMESPACE) - .put(MetricsTag.TASK_QUEUE, testWorkflowRule.getTaskQueue()) - .build(); - reporter.assertGauge(MetricsType.RESOURCE_MEM_USAGE, nsAndTaskQueue, (val) -> val > 0); - reporter.assertGauge( - MetricsType.RESOURCE_CPU_USAGE, - nsAndTaskQueue, - (val) -> { - // CPU use can be so low as to be 0, so can't really make any assertion here. - return true; - }); - reporter.assertGauge(MetricsType.RESOURCE_MEM_PID, nsAndTaskQueue, (val) -> true); - reporter.assertGauge(MetricsType.RESOURCE_CPU_PID, nsAndTaskQueue, (val) -> true); - } - - @Category(IndependentResourceBasedTests.class) - @Test(timeout = 300 * 1000) - public void canRunHeavyMemoryWithResourceBasedTuner() { - ResourceTunerWorkflow workflow = testWorkflowRule.newWorkflowStub(ResourceTunerWorkflow.class); - workflow.execute(100, 30000000); - } - - @WorkflowInterface - public interface ResourceTunerWorkflow { - @WorkflowMethod - String execute(int numActivities, int memCeiling); - } - - public static class ResourceTunerWorkflowImpl implements ResourceTunerWorkflow { - @Override - public String execute(int numActivities, int memCeiling) { - SleepActivity activity = - Workflow.newActivityStub( - SleepActivity.class, - ActivityOptions.newBuilder() - .setStartToCloseTimeout(Duration.ofMinutes(1)) - .setHeartbeatTimeout(Duration.ofSeconds(20)) - .build()); - - List> promises = new ArrayList<>(); - for (int j = 0; j < numActivities; j++) { - Promise promise = Async.procedure(activity::useResources, memCeiling); - promises.add(promise); - } - - for (Promise promise : promises) { - promise.get(); - } - - return "I'm done"; - } - } - - @ActivityInterface - public interface SleepActivity { - void useResources(int memCeiling); - } - - public static class ActivitiesImpl implements SleepActivity { - @Override - public void useResources(int memCeiling) { - try { - int randNumBytes = (int) (Math.random() * memCeiling); - @SuppressWarnings("unused") - byte[] bytes = new byte[randNumBytes]; - // Need to wait at least a second to give metrics a chance to be reported - // (and also simulate some actual work in the activity) - Thread.sleep(1100); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - } -} diff --git a/temporal-sdk/src/test/java/io/temporal/worker/WorkerOptionsTest.java b/temporal-sdk/src/test/java/io/temporal/worker/WorkerOptionsTest.java index 7b051de07..934e014cd 100644 --- a/temporal-sdk/src/test/java/io/temporal/worker/WorkerOptionsTest.java +++ b/temporal-sdk/src/test/java/io/temporal/worker/WorkerOptionsTest.java @@ -21,9 +21,7 @@ package io.temporal.worker; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import io.temporal.worker.tuning.*; import org.junit.Test; public class WorkerOptionsTest { @@ -49,49 +47,4 @@ public void verifyWorkerOptionsEquality() { WorkerOptions w2 = WorkerOptions.newBuilder().build(); assertEquals(w1, w2); } - - @Test - public void canBuildMixedSlotSupplierTuner() { - ResourceBasedController resourceController = - ResourceBasedController.newSystemInfoController( - ResourceBasedControllerOptions.newBuilder(0.5, 0.5).build()); - - SlotSupplier workflowTaskSlotSupplier = new FixedSizeSlotSupplier<>(10); - SlotSupplier activityTaskSlotSupplier = - ResourceBasedSlotSupplier.createForActivity( - resourceController, ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS); - SlotSupplier localActivitySlotSupplier = - ResourceBasedSlotSupplier.createForLocalActivity( - resourceController, ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS); - - WorkerOptions.newBuilder() - .setWorkerTuner( - new CompositeTuner( - workflowTaskSlotSupplier, activityTaskSlotSupplier, localActivitySlotSupplier)) - .build(); - } - - @Test - public void throwsIfResourceControllerIsNotSame() { - ResourceBasedController resourceController1 = - ResourceBasedController.newSystemInfoController( - ResourceBasedControllerOptions.newBuilder(0.5, 0.5).build()); - ResourceBasedController resourceController2 = - ResourceBasedController.newSystemInfoController( - ResourceBasedControllerOptions.newBuilder(0.2, 0.3).build()); - - SlotSupplier workflowTaskSlotSupplier = new FixedSizeSlotSupplier<>(10); - SlotSupplier activityTaskSlotSupplier = - ResourceBasedSlotSupplier.createForActivity( - resourceController1, ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS); - SlotSupplier localActivitySlotSupplier = - ResourceBasedSlotSupplier.createForLocalActivity( - resourceController2, ResourceBasedTuner.DEFAULT_ACTIVITY_SLOT_OPTIONS); - - assertThrows( - IllegalArgumentException.class, - () -> - new CompositeTuner( - workflowTaskSlotSupplier, activityTaskSlotSupplier, localActivitySlotSupplier)); - } }