From edf2211c086b839aa1bb1957245febc681685e2e Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Mon, 16 Oct 2023 23:54:31 -0400 Subject: [PATCH 01/68] Implement local resources for workers (#1282) --- _site/docs/contribute/design-documents.md | 3 +- .../buildfarm/common/config/Backplane.java | 15 +- .../common/config/LimitedResource.java | 43 ++++ .../build/buildfarm/common/config/Worker.java | 6 + .../common/redis/BalancedRedisQueue.java | 11 + .../instance/shard/OperationQueue.java | 84 +++++++- src/main/java/build/buildfarm/worker/BUILD | 1 + .../worker/DequeueMatchEvaluator.java | 37 ++-- .../java/build/buildfarm/worker/Executor.java | 3 + .../build/buildfarm/worker/WorkerContext.java | 2 + .../build/buildfarm/worker/resources/BUILD | 1 + .../worker/resources/LocalResourceSet.java | 37 ++++ .../resources/LocalResourceSetMetrics.java | 46 +++++ .../resources/LocalResourceSetUtils.java | 120 +++++++++++ .../worker/shard/ShardWorkerContext.java | 22 +- .../build/buildfarm/worker/shard/Worker.java | 2 + .../common/config/GrpcMetricsTest.java | 1 - .../java/build/buildfarm/instance/shard/BUILD | 192 +++++++++++++++++- .../shard/RedisShardBackplaneTest.java | 6 +- .../instance/shard/ShardInstanceTest.java | 1 - .../worker/DequeueMatchEvaluatorTest.java | 168 ++++++++++++++- .../buildfarm/worker/StubWorkerContext.java | 5 + .../java/build/buildfarm/worker/shard/BUILD | 1 + .../worker/shard/ShardWorkerContextTest.java | 2 + 24 files changed, 759 insertions(+), 50 deletions(-) create mode 100644 src/main/java/build/buildfarm/common/config/LimitedResource.java create mode 100644 src/main/java/build/buildfarm/worker/resources/LocalResourceSet.java create mode 100644 src/main/java/build/buildfarm/worker/resources/LocalResourceSetMetrics.java create mode 100644 src/main/java/build/buildfarm/worker/resources/LocalResourceSetUtils.java diff --git a/_site/docs/contribute/design-documents.md b/_site/docs/contribute/design-documents.md index f7b1c0b80..ea608696f 100644 --- a/_site/docs/contribute/design-documents.md +++ b/_site/docs/contribute/design-documents.md @@ -5,4 +5,5 @@ parent: Contribute nav_order: 2 --- -[Infinite Cache (Storage Workers)](https://docs.google.com/document/d/1IQQbWPzjSluDL25FZ9ADtNIOT90PLijQGIAC4RbwMjY/edit?usp=sharing) \ No newline at end of file +[Infinite Cache (Storage Workers)](https://docs.google.com/document/d/1IQQbWPzjSluDL25FZ9ADtNIOT90PLijQGIAC4RbwMjY/edit?usp=sharing) +[Local and Global Resources](https://docs.google.com/document/d/1u0TkmVmdMS53PWR1hgh-a_cj3NmQYE0Favv9aGFfQZs/edit?usp=sharing) \ No newline at end of file diff --git a/src/main/java/build/buildfarm/common/config/Backplane.java b/src/main/java/build/buildfarm/common/config/Backplane.java index 5c21532ab..cc15a6028 100644 --- a/src/main/java/build/buildfarm/common/config/Backplane.java +++ b/src/main/java/build/buildfarm/common/config/Backplane.java @@ -1,6 +1,7 @@ package build.buildfarm.common.config; -import com.google.common.base.Strings; +import java.util.ArrayList; +import java.util.List; import lombok.AccessLevel; import lombok.Data; import lombok.Getter; @@ -52,13 +53,7 @@ public enum BACKPLANE_TYPE { private boolean cacheCas = false; private long priorityPollIntervalMillis = 100; - public String getRedisUri() { - // use environment override (useful for containerized deployment) - if (!Strings.isNullOrEmpty(System.getenv("REDIS_URI"))) { - return System.getenv("REDIS_URI"); - } - - // use configured value - return redisUri; - } + // These limited resources are shared across all workers. + // An example would be a limited number of seats to a license server. + private List resources = new ArrayList<>(); } diff --git a/src/main/java/build/buildfarm/common/config/LimitedResource.java b/src/main/java/build/buildfarm/common/config/LimitedResource.java new file mode 100644 index 000000000..f3b09ff62 --- /dev/null +++ b/src/main/java/build/buildfarm/common/config/LimitedResource.java @@ -0,0 +1,43 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.common.config; + +import lombok.Data; + +/** + * @class Limited Resource + * @brief A fixed amount of a specific resource. + * @details We define a limited resource as a counting semaphore whose configuration contains a name + * and a count representing a physical or logical group of units obtained by executors as a + * precondition to fulfill a long running operation. These units are released upon the + * operation's completion. The resource is requested by the action's platform properties. + */ +@Data +public class LimitedResource { + /** + * @field name + * @brief The name of the resource. + * @details This should correspond to the platform property's key name: + * resources:: + */ + public String name; + + /** + * @field amount + * @brief The total amount of the resource that's available for use during execution. + * @details As a counting semaphore, this amount becomes the limit. + */ + public int amount = 1; +} diff --git a/src/main/java/build/buildfarm/common/config/Worker.java b/src/main/java/build/buildfarm/common/config/Worker.java index 1661df9bf..e987c1e37 100644 --- a/src/main/java/build/buildfarm/common/config/Worker.java +++ b/src/main/java/build/buildfarm/common/config/Worker.java @@ -5,6 +5,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import javax.naming.ConfigurationException; @@ -39,6 +40,11 @@ public class Worker { private ExecutionPolicy[] executionPolicies = {}; private SandboxSettings sandboxSettings = new SandboxSettings(); + // These limited resources are only for the individual worker. + // An example would be hardware resources such as GPUs. + // If you want GPU actions to run exclusively, define a single GPU resource. + private List resources = new ArrayList<>(); + public ExecutionPolicy[] getExecutionPolicies() { if (executionPolicies != null) { return executionPolicies; diff --git a/src/main/java/build/buildfarm/common/redis/BalancedRedisQueue.java b/src/main/java/build/buildfarm/common/redis/BalancedRedisQueue.java index 52f937c8e..9123f5eaf 100644 --- a/src/main/java/build/buildfarm/common/redis/BalancedRedisQueue.java +++ b/src/main/java/build/buildfarm/common/redis/BalancedRedisQueue.java @@ -236,6 +236,17 @@ public String dequeue(JedisCluster jedis) throws InterruptedException { } } + /** + * @brief Pop element into internal dequeue and return value. + * @details Null is returned if the queue is empty. + * @return The value of the transfered element. null if queue is empty or thread was interrupted. + * @note Suggested return identifier: val. + */ + public String nonBlockingDequeue(JedisCluster jedis) throws InterruptedException { + QueueInterface queue = queues.get(roundRobinPopIndex()); + return queue.nonBlockingDequeue(jedis); + } + /** * @brief Get the current pop queue. * @details Get the queue that the balanced queue intends to pop from next. diff --git a/src/main/java/build/buildfarm/instance/shard/OperationQueue.java b/src/main/java/build/buildfarm/instance/shard/OperationQueue.java index 64f1f2bef..78dd77c76 100644 --- a/src/main/java/build/buildfarm/instance/shard/OperationQueue.java +++ b/src/main/java/build/buildfarm/instance/shard/OperationQueue.java @@ -24,6 +24,7 @@ import com.google.common.collect.SetMultimap; import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; import redis.clients.jedis.JedisCluster; /** @@ -48,6 +49,14 @@ public class OperationQueue { */ private final List queues; + /** + * @field currentDequeueIndex + * @brief The current queue index to dequeue from. + * @details Used in a round-robin fashion to ensure an even distribution of dequeues across + * matched queues. + */ + private int currentDequeueIndex = 0; + /** * @brief Constructor. * @details Construct the operation queue with various provisioned redis queues. @@ -186,8 +195,18 @@ public void push( */ public String dequeue(JedisCluster jedis, List provisions) throws InterruptedException { - BalancedRedisQueue queue = chooseEligibleQueue(provisions); - return queue.dequeue(jedis); + // Select all matched queues, and attempt dequeuing via round-robin. + List queues = chooseEligibleQueues(provisions); + int index = roundRobinPopIndex(queues); + String value = queues.get(index).nonBlockingDequeue(jedis); + + // Keep iterating over matched queues until we find one that is non-empty and provides a + // dequeued value. + while (value == null) { + index = roundRobinPopIndex(queues); + value = queues.get(index).nonBlockingDequeue(jedis); + } + return value; } /** @@ -270,6 +289,39 @@ private BalancedRedisQueue chooseEligibleQueue(List provision } } + throwNoEligibleQueueException(provisions); + return null; + } + + /** + * @brief Choose an eligible queues based on given properties. + * @details We use the platform execution properties of a queue entry to determine the appropriate + * queues. If there no eligible queues, an exception is thrown. + * @param provisions Provisions to check that requirements are met. + * @return The chosen queues. + * @note Suggested return identifier: queues. + */ + private List chooseEligibleQueues(List provisions) { + List eligibleQueues = + queues.stream() + .filter(provisionedQueue -> provisionedQueue.isEligible(toMultimap(provisions))) + .map(provisionedQueue -> provisionedQueue.queue()) + .collect(Collectors.toList()); + + if (eligibleQueues.isEmpty()) { + throwNoEligibleQueueException(provisions); + } + + return eligibleQueues; + } + + /** + * @brief Throw an exception that explains why there are no eligible queues. + * @details This function should only be called, when there were no matched queues. + * @param provisions Provisions to check that requirements are met. + * @return no return. + */ + private void throwNoEligibleQueueException(List provisions) { // At this point, we were unable to match an action to an eligible queue. // We will build an error explaining why the matching failed. This will help user's properly // configure their queue or adjust the execution_properties of their actions. @@ -286,6 +338,34 @@ private BalancedRedisQueue chooseEligibleQueue(List provision + eligibilityResults); } + /** + * @brief Get the current queue index for round-robin dequeues. + * @details Adjusts the round-robin index for next call. + * @param matchedQueues The queues to round robin. + * @return The current round-robin index. + * @note Suggested return identifier: queueIndex. + */ + private int roundRobinPopIndex(List matchedQueues) { + int currentIndex = currentDequeueIndex; + currentDequeueIndex = nextQueueInRoundRobin(currentDequeueIndex, matchedQueues); + return currentIndex; + } + + /** + * @brief Get the next queue in the round robin. + * @details If we are currently on the last queue it becomes the first queue. + * @param index Current queue index. + * @param matchedQueues The queues to round robin. + * @return And adjusted val based on the current queue index. + * @note Suggested return identifier: adjustedCurrentQueue. + */ + private int nextQueueInRoundRobin(int index, List matchedQueues) { + if (index >= matchedQueues.size() - 1) { + return 0; + } + return index + 1; + } + /** * @brief Convert proto provisions into java multimap. * @details This conversion is done to more easily check if a key/value exists in the provisions. diff --git a/src/main/java/build/buildfarm/worker/BUILD b/src/main/java/build/buildfarm/worker/BUILD index c25b9e74d..e24b50d73 100644 --- a/src/main/java/build/buildfarm/worker/BUILD +++ b/src/main/java/build/buildfarm/worker/BUILD @@ -36,6 +36,7 @@ java_library( "@maven//:io_grpc_grpc_stub", "@maven//:io_prometheus_simpleclient", "@maven//:org_apache_commons_commons_compress", + "@maven//:org_apache_commons_commons_lang3", "@maven//:org_jetbrains_annotations", "@maven//:org_projectlombok_lombok", ], diff --git a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java index c3c4c1dfc..3536ed6ba 100644 --- a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java +++ b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java @@ -14,11 +14,12 @@ package build.buildfarm.worker; -import build.bazel.remote.execution.v2.Command; import build.bazel.remote.execution.v2.Platform; import build.buildfarm.common.ExecutionProperties; import build.buildfarm.common.config.BuildfarmConfigs; import build.buildfarm.v1test.QueueEntry; +import build.buildfarm.worker.resources.LocalResourceSet; +import build.buildfarm.worker.resources.LocalResourceSetUtils; import com.google.common.collect.Iterables; import com.google.common.collect.SetMultimap; import org.jetbrains.annotations.NotNull; @@ -45,6 +46,7 @@ public class DequeueMatchEvaluator { * @brief Decide whether the worker should keep the operation or put it back on the queue. * @details Compares the platform properties of the worker to the operation's platform properties. * @param workerProvisions The provisions of the worker. + * @param resourceSet The limited resources that the worker has available. * @param queueEntry An entry recently removed from the queue. * @return Whether or not the worker should accept or reject the queue entry. * @note Overloaded. @@ -53,24 +55,10 @@ public class DequeueMatchEvaluator { @SuppressWarnings("NullableProblems") @NotNull public static boolean shouldKeepOperation( - SetMultimap workerProvisions, QueueEntry queueEntry) { - return shouldKeepViaPlatform(workerProvisions, queueEntry.getPlatform()); - } - - /** - * @brief Decide whether the worker should keep the operation or put it back on the queue. - * @details Compares the platform properties of the worker to the operation's platform properties. - * @param workerProvisions The provisions of the worker. - * @param command A command to evaluate. - * @return Whether or not the worker should accept or reject the queue entry. - * @note Overloaded. - * @note Suggested return identifier: shouldKeepOperation. - */ - @SuppressWarnings("NullableProblems") - @NotNull - public static boolean shouldKeepOperation( - SetMultimap workerProvisions, Command command) { - return shouldKeepViaPlatform(workerProvisions, command.getPlatform()); + SetMultimap workerProvisions, + LocalResourceSet resourceSet, + QueueEntry queueEntry) { + return shouldKeepViaPlatform(workerProvisions, resourceSet, queueEntry.getPlatform()); } /** @@ -79,6 +67,7 @@ public static boolean shouldKeepOperation( * @details Compares the platform properties of the worker to the platform properties of the * operation. * @param workerProvisions The provisions of the worker. + * @param resourceSet The limited resources that the worker has available. * @param platform The platforms of operation. * @return Whether or not the worker should accept or reject the operation. * @note Suggested return identifier: shouldKeepOperation. @@ -86,9 +75,15 @@ public static boolean shouldKeepOperation( @SuppressWarnings("NullableProblems") @NotNull private static boolean shouldKeepViaPlatform( - SetMultimap workerProvisions, Platform platform) { - // attempt to execute everything the worker gets off the queue. + SetMultimap workerProvisions, + LocalResourceSet resourceSet, + Platform platform) { + // attempt to execute everything the worker gets off the queue, + // provided there is enough resources to do so. // this is a recommended configuration. + if (!LocalResourceSetUtils.claimResources(platform, resourceSet)) { + return false; + } if (configs.getWorker().getDequeueMatchSettings().isAcceptEverything()) { return true; } diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index 0416c2354..f4776a6f0 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -359,6 +359,9 @@ public void run(ResourceLimits limits) { } finally { boolean wasInterrupted = Thread.interrupted(); try { + // Now that the execution has finished we can return any of the claims against local + // resources. + workerContext.returnLocalResources(operationContext.queueEntry); owner.releaseExecutor( operationName, limits.cpu.claimed, diff --git a/src/main/java/build/buildfarm/worker/WorkerContext.java b/src/main/java/build/buildfarm/worker/WorkerContext.java index 873ad1b93..70060acea 100644 --- a/src/main/java/build/buildfarm/worker/WorkerContext.java +++ b/src/main/java/build/buildfarm/worker/WorkerContext.java @@ -130,4 +130,6 @@ IOResource limitExecution( int commandExecutionClaims(Command command); ResourceLimits commandExecutionSettings(Command command); + + void returnLocalResources(QueueEntry queueEntry); } diff --git a/src/main/java/build/buildfarm/worker/resources/BUILD b/src/main/java/build/buildfarm/worker/resources/BUILD index ac2d69179..36a4dc3d3 100644 --- a/src/main/java/build/buildfarm/worker/resources/BUILD +++ b/src/main/java/build/buildfarm/worker/resources/BUILD @@ -8,6 +8,7 @@ java_library( "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "@maven//:com_google_guava_guava", "@maven//:com_googlecode_json_simple_json_simple", + "@maven//:io_prometheus_simpleclient", "@maven//:org_apache_commons_commons_lang3", ], ) diff --git a/src/main/java/build/buildfarm/worker/resources/LocalResourceSet.java b/src/main/java/build/buildfarm/worker/resources/LocalResourceSet.java new file mode 100644 index 000000000..97d64fe9b --- /dev/null +++ b/src/main/java/build/buildfarm/worker/resources/LocalResourceSet.java @@ -0,0 +1,37 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.worker.resources; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Semaphore; + +/** + * @class Local Resource Set + * @brief A fixed amount of a specific resource. + * @details We define limited resources as a counting semaphores whose configuration contains a name + * and a count representing a physical or logical group of units obtained by executors as a + * precondition to fulfill a long running operation. These units are released upon the + * operation's completion. The resource is requested by the action's platform properties. These + * resources are specific to the individual worker. + */ +public class LocalResourceSet { + /** + * @field resources + * @brief A set containing resource semaphores organized by name. + * @details Key is name, and value is current usage amount. + */ + public Map resources = new HashMap<>(); +} diff --git a/src/main/java/build/buildfarm/worker/resources/LocalResourceSetMetrics.java b/src/main/java/build/buildfarm/worker/resources/LocalResourceSetMetrics.java new file mode 100644 index 000000000..787a7840e --- /dev/null +++ b/src/main/java/build/buildfarm/worker/resources/LocalResourceSetMetrics.java @@ -0,0 +1,46 @@ +// Copyright 2020 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.worker.resources; + +import io.prometheus.client.Gauge; + +/** + * @class LocalResourceSetMetrics + * @brief Tracks metrics related to a worker's limited local resources. + * @details Answers how many resources exist, how many are claimed, and by how many requesters. + */ +public class LocalResourceSetMetrics { + public static final Gauge resourceUsageMetric = + Gauge.build() + .name("local_resource_usage") + .labelNames("resource_name") + .help("The number of claims for each resource currently being used for execution") + .register(); + + public static final Gauge resourceTotalMetric = + Gauge.build() + .name("local_resource_total") + .labelNames("resource_name") + .help("The total number of claims exist for a particular resource") + .register(); + + public static final Gauge requestersMetric = + Gauge.build() + .name("local_resource_requesters") + .labelNames("resource_name") + .help( + "Tracks how many actions have requested local resources. This can help determine if resources are being hogged by some actions.") + .register(); +} diff --git a/src/main/java/build/buildfarm/worker/resources/LocalResourceSetUtils.java b/src/main/java/build/buildfarm/worker/resources/LocalResourceSetUtils.java new file mode 100644 index 000000000..e8a1e5613 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/resources/LocalResourceSetUtils.java @@ -0,0 +1,120 @@ +// Copyright 2020 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.worker.resources; + +import build.bazel.remote.execution.v2.Platform; +import build.buildfarm.common.config.LimitedResource; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Semaphore; +import org.apache.commons.lang3.StringUtils; + +/** + * @class LocalResourceSetUtils + * @brief Utilities for working with the worker's set of local limited resources. + * @details The methods help with allocation / de-allocation of claims, as well as metrics printing. + */ +public class LocalResourceSetUtils { + private static final LocalResourceSetMetrics metrics = new LocalResourceSetMetrics(); + + public static LocalResourceSet create(List resources) { + LocalResourceSet resourceSet = new LocalResourceSet(); + for (LimitedResource resource : resources) { + resourceSet.resources.put(resource.getName(), new Semaphore(resource.getAmount())); + metrics.resourceTotalMetric.labels(resource.getName()).set(resource.getAmount()); + } + return resourceSet; + } + + public static boolean claimResources(Platform platform, LocalResourceSet resourceSet) { + List> claimed = new ArrayList<>(); + + boolean allClaimed = true; + for (Platform.Property property : platform.getPropertiesList()) { + // Skip properties that are not requesting a limited resource. + String resourceName = getResourceName(property); + Semaphore resource = resourceSet.resources.get(resourceName); + if (resource == null) { + continue; + } + + // Attempt to claim. If claiming fails, we must return all other claims. + int requestAmount = getResourceRequestAmount(property); + boolean wasAcquired = semaphoreAquire(resource, resourceName, requestAmount); + if (wasAcquired) { + claimed.add(new AbstractMap.SimpleEntry<>(resourceName, requestAmount)); + } else { + allClaimed = false; + break; + } + } + + // cleanup remaining resources if they were not all claimed. + if (!allClaimed) { + for (Map.Entry claim : claimed) { + semaphoreRelease( + resourceSet.resources.get(claim.getKey()), claim.getKey(), claim.getValue()); + } + } + + return allClaimed; + } + + public static void releaseClaims(Platform platform, LocalResourceSet resourceSet) { + for (Platform.Property property : platform.getPropertiesList()) { + String resourceName = getResourceName(property); + Semaphore resource = resourceSet.resources.get(resourceName); + if (resource == null) { + continue; + } + int requestAmount = getResourceRequestAmount(property); + semaphoreRelease(resource, resourceName, requestAmount); + } + } + + private static boolean semaphoreAquire(Semaphore resource, String resourceName, int amount) { + boolean wasAcquired = resource.tryAcquire(amount); + if (wasAcquired) { + metrics.resourceUsageMetric.labels(resourceName).inc(amount); + } + metrics.requestersMetric.labels(resourceName).inc(); + return wasAcquired; + } + + private static void semaphoreRelease(Semaphore resource, String resourceName, int amount) { + resource.release(amount); + metrics.resourceUsageMetric.labels(resourceName).dec(amount); + metrics.requestersMetric.labels(resourceName).dec(); + } + + private static int getResourceRequestAmount(Platform.Property property) { + // We support resource values that are not numbers and interpret them as a request for 1 + // resource. For example "gpu:RTX-4090" is equivalent to resource:gpu:1". + try { + return Integer.parseInt(property.getValue()); + } catch (NumberFormatException e) { + return 1; + } + } + + private static String getResourceName(Platform.Property property) { + // We match to keys whether they are prefixed "resource:" or not. + // "resource:gpu:1" requests the gpu resource in the same way that "gpu:1" does. + // The prefix originates from bazel's syntax for the --extra_resources flag. + return StringUtils.removeStart(property.getName(), "resource:"); + } +} diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index d32731879..c844f5396 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -60,6 +60,8 @@ import build.buildfarm.worker.cgroup.Cpu; import build.buildfarm.worker.cgroup.Group; import build.buildfarm.worker.cgroup.Mem; +import build.buildfarm.worker.resources.LocalResourceSet; +import build.buildfarm.worker.resources.LocalResourceSetUtils; import build.buildfarm.worker.resources.ResourceDecider; import build.buildfarm.worker.resources.ResourceLimits; import com.google.common.annotations.VisibleForTesting; @@ -129,6 +131,7 @@ class ShardWorkerContext implements WorkerContext { private final Group operationsGroup = executionsGroup.getChild("operations"); private final CasWriter writer; private final boolean errorOperationRemainingResources; + private final LocalResourceSet resourceSet; static SetMultimap getMatchProvisions( Iterable policies, int executeStageWidth) { @@ -162,6 +165,7 @@ static SetMultimap getMatchProvisions( boolean onlyMulticoreTests, boolean allowBringYourOwnContainer, boolean errorOperationRemainingResources, + LocalResourceSet resourceSet, CasWriter writer) { this.name = name; this.matchProvisions = getMatchProvisions(policies, executeStageWidth); @@ -182,6 +186,7 @@ static SetMultimap getMatchProvisions( this.onlyMulticoreTests = onlyMulticoreTests; this.allowBringYourOwnContainer = allowBringYourOwnContainer; this.errorOperationRemainingResources = errorOperationRemainingResources; + this.resourceSet = resourceSet; this.writer = writer; } @@ -273,6 +278,12 @@ public QueuedOperation getQueuedOperation(QueueEntry queueEntry) @SuppressWarnings("ConstantConditions") private void matchInterruptible(MatchListener listener) throws IOException, InterruptedException { + QueueEntry queueEntry = takeEntryOffOperationQueue(listener); + decideWhetherToKeepOperation(queueEntry, listener); + } + + private QueueEntry takeEntryOffOperationQueue(MatchListener listener) + throws IOException, InterruptedException { listener.onWaitStart(); QueueEntry queueEntry = null; try { @@ -294,9 +305,13 @@ private void matchInterruptible(MatchListener listener) throws IOException, Inte // transient backplane errors will propagate a null queueEntry } listener.onWaitEnd(); + return queueEntry; + } + private void decideWhetherToKeepOperation(QueueEntry queueEntry, MatchListener listener) + throws IOException, InterruptedException { if (queueEntry == null - || DequeueMatchEvaluator.shouldKeepOperation(matchProvisions, queueEntry)) { + || DequeueMatchEvaluator.shouldKeepOperation(matchProvisions, resourceSet, queueEntry)) { listener.onEntry(queueEntry); } else { backplane.rejectOperation(queueEntry); @@ -306,6 +321,11 @@ private void matchInterruptible(MatchListener listener) throws IOException, Inte } } + @Override + public void returnLocalResources(QueueEntry queueEntry) { + LocalResourceSetUtils.releaseClaims(queueEntry.getPlatform(), resourceSet); + } + @Override public void match(MatchListener listener) throws InterruptedException { RetryingMatchListener dedupMatchListener = diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index 153df1a37..7a0d4a60e 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -59,6 +59,7 @@ import build.buildfarm.worker.PutOperationStage; import build.buildfarm.worker.ReportResultStage; import build.buildfarm.worker.SuperscalarPipelineStage; +import build.buildfarm.worker.resources.LocalResourceSetUtils; import com.google.common.cache.LoadingCache; import com.google.common.collect.Lists; import com.google.longrunning.Operation; @@ -562,6 +563,7 @@ public void start() throws ConfigurationException, InterruptedException, IOExcep configs.getWorker().isOnlyMulticoreTests(), configs.getWorker().isAllowBringYourOwnContainer(), configs.getWorker().isErrorOperationRemainingResources(), + LocalResourceSetUtils.create(configs.getWorker().getResources()), writer); pipeline = new Pipeline(); diff --git a/src/test/java/build/buildfarm/common/config/GrpcMetricsTest.java b/src/test/java/build/buildfarm/common/config/GrpcMetricsTest.java index 5301b8260..547f81b2c 100644 --- a/src/test/java/build/buildfarm/common/config/GrpcMetricsTest.java +++ b/src/test/java/build/buildfarm/common/config/GrpcMetricsTest.java @@ -30,7 +30,6 @@ public void testHandleGrpcMetricIntercepts_withLatencyBucket() { grpcMetrics.setEnabled(true); grpcMetrics.setProvideLatencyHistograms(true); grpcMetrics.setLatencyBuckets(new double[] {1, 2, 3}); - GrpcMetrics.handleGrpcMetricIntercepts(serverBuilder, grpcMetrics); verify(serverBuilder, times(1)).intercept(any(MonitoringServerInterceptor.class)); } diff --git a/src/test/java/build/buildfarm/instance/shard/BUILD b/src/test/java/build/buildfarm/instance/shard/BUILD index 6a501a1b6..cd0ae18d2 100644 --- a/src/test/java/build/buildfarm/instance/shard/BUILD +++ b/src/test/java/build/buildfarm/instance/shard/BUILD @@ -1,7 +1,195 @@ java_test( - name = "tests", + name = "DispatchedMonitorTest", size = "small", - srcs = glob(["*.java"]), + srcs = [ + "DispatchedMonitorTest.java", + "UnobservableWatcher.java", + ], + data = ["//examples:example_configs"], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/actioncache", + "//src/main/java/build/buildfarm/backplane", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/server", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@googleapis//:google_longrunning_operations_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", + "@maven//:com_github_ben_manes_caffeine_caffeine", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:org_mockito_mockito_core", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + +java_test( + name = "RedisShardBackplaneTest", + size = "small", + srcs = [ + "RedisShardBackplaneTest.java", + "UnobservableWatcher.java", + ], + data = ["//examples:example_configs"], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/actioncache", + "//src/main/java/build/buildfarm/backplane", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/server", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@googleapis//:google_longrunning_operations_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", + "@maven//:com_github_ben_manes_caffeine_caffeine", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:org_mockito_mockito_core", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + +java_test( + name = "RedisShardSubscriberTest", + size = "small", + srcs = [ + "RedisShardSubscriberTest.java", + "UnobservableWatcher.java", + ], + data = ["//examples:example_configs"], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/actioncache", + "//src/main/java/build/buildfarm/backplane", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/server", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@googleapis//:google_longrunning_operations_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", + "@maven//:com_github_ben_manes_caffeine_caffeine", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:org_mockito_mockito_core", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + +java_test( + name = "ShardInstanceTest", + size = "small", + srcs = [ + "ShardInstanceTest.java", + "UnobservableWatcher.java", + ], + data = ["//examples:example_configs"], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/actioncache", + "//src/main/java/build/buildfarm/backplane", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/server", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@googleapis//:google_longrunning_operations_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", + "@maven//:com_github_ben_manes_caffeine_caffeine", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:org_mockito_mockito_core", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + +java_test( + name = "TimedWatcherTest", + size = "small", + srcs = [ + "TimedWatcherTest.java", + "UnobservableWatcher.java", + ], + data = ["//examples:example_configs"], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/actioncache", + "//src/main/java/build/buildfarm/backplane", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/server", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@googleapis//:google_longrunning_operations_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", + "@maven//:com_github_ben_manes_caffeine_caffeine", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:org_mockito_mockito_core", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + +java_test( + name = "UtilTest", + size = "small", + srcs = [ + "UnobservableWatcher.java", + "UtilTest.java", + ], data = ["//examples:example_configs"], test_class = "build.buildfarm.AllTests", deps = [ diff --git a/src/test/java/build/buildfarm/instance/shard/RedisShardBackplaneTest.java b/src/test/java/build/buildfarm/instance/shard/RedisShardBackplaneTest.java index 16e5bb8fb..3772459cf 100644 --- a/src/test/java/build/buildfarm/instance/shard/RedisShardBackplaneTest.java +++ b/src/test/java/build/buildfarm/instance/shard/RedisShardBackplaneTest.java @@ -213,8 +213,7 @@ public void dispatchedOperationsShowProperRequeueAmount0to1() .setRequeueAttempts(STARTING_REQUEUE_AMOUNT) .build(); String queueEntryJson = JsonFormat.printer().print(queueEntry); - when(jedisCluster.brpoplpush(any(String.class), any(String.class), any(Integer.class))) - .thenReturn(queueEntryJson); + when(jedisCluster.rpoplpush(any(String.class), any(String.class))).thenReturn(queueEntryJson); // PRE-ASSERT when(jedisCluster.hsetnx(any(String.class), any(String.class), any(String.class))) @@ -266,8 +265,7 @@ public void dispatchedOperationsShowProperRequeueAmount1to2() .setRequeueAttempts(STARTING_REQUEUE_AMOUNT) .build(); String queueEntryJson = JsonFormat.printer().print(queueEntry); - when(jedisCluster.brpoplpush(any(String.class), any(String.class), any(Integer.class))) - .thenReturn(queueEntryJson); + when(jedisCluster.rpoplpush(any(String.class), any(String.class))).thenReturn(queueEntryJson); // PRE-ASSERT when(jedisCluster.hsetnx(any(String.class), any(String.class), any(String.class))) diff --git a/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java b/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java index 1fdb7fc46..aa1adc5d3 100644 --- a/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java +++ b/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java @@ -1052,7 +1052,6 @@ public void containsBlobReflectsWorkerWithUnknownSize() throws Exception { @Test public void findMissingBlobsTest_ViaBackPlane() throws Exception { - Set activeWorkers = ImmutableSet.of("worker1", "worker2", "worker3"); Set expiredWorkers = ImmutableSet.of("workerX", "workerY", "workerZ"); Set imposterWorkers = ImmutableSet.of("imposter1", "imposter2", "imposter3"); diff --git a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java index bc7acd00a..47f25cdf7 100644 --- a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java +++ b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java @@ -19,8 +19,10 @@ import build.bazel.remote.execution.v2.Platform; import build.buildfarm.common.config.BuildfarmConfigs; import build.buildfarm.v1test.QueueEntry; +import build.buildfarm.worker.resources.LocalResourceSet; import com.google.common.collect.HashMultimap; import com.google.common.collect.SetMultimap; +import java.util.concurrent.Semaphore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -52,10 +54,12 @@ public class DequeueMatchEvaluatorTest { public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); QueueEntry entry = QueueEntry.newBuilder().setPlatform(Platform.newBuilder()).build(); // ACT - boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -70,6 +74,7 @@ public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); workerProvisions.put("cores", "11"); QueueEntry entry = @@ -81,7 +86,8 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { .build(); // ACT - boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because it has more cores than the min-cores requested @@ -98,6 +104,7 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ARRANGE configs.getWorker().getDequeueMatchSettings().setAcceptEverything(false); SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); workerProvisions.put("cores", "10"); QueueEntry entry = @@ -109,7 +116,8 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { .build(); // ACT - boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because it has less cores than the min-cores requested @@ -123,6 +131,7 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Exception { // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); workerProvisions.put("cores", "10"); QueueEntry entry = @@ -136,7 +145,8 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti .build(); // ACT - boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because it has the same cores as the min-cores requested @@ -153,6 +163,7 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAcceptEverything(false); configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(false); SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); QueueEntry entry = QueueEntry.newBuilder() @@ -163,7 +174,8 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E .build(); // ACT - boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isFalse(); @@ -172,7 +184,7 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -181,9 +193,151 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, entry); + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); } + + // Function under test: shouldKeepOperation + // Reason for testing: the local resource should be claimed + // Failure explanation: semaphore claim did not work as expected. + @Test + public void shouldKeepOperationClaimsResource() throws Exception { + // ARRANGE + configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); + configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); + SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); + resourceSet.resources.put("FOO", new Semaphore(1)); + + QueueEntry entry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties( + Platform.Property.newBuilder().setName("resource:FOO").setValue("1"))) + .build(); + + // PRE-ASSERT + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); + + // ACT + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker accepts because the resource is available. + assertThat(shouldKeep).isTrue(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); + + // ACT + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker rejects because there are no resources left. + assertThat(shouldKeep).isFalse(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); + } + + // Function under test: shouldKeepOperation + // Reason for testing: the local resources should be claimed + // Failure explanation: semaphore claim did not work as expected. + @Test + public void shouldKeepOperationClaimsMultipleResource() throws Exception { + // ARRANGE + configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); + configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); + SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); + resourceSet.resources.put("FOO", new Semaphore(2)); + resourceSet.resources.put("BAR", new Semaphore(4)); + + QueueEntry entry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties( + Platform.Property.newBuilder().setName("resource:FOO").setValue("1")) + .addProperties( + Platform.Property.newBuilder().setName("resource:BAR").setValue("2"))) + .build(); + + // PRE-ASSERT + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(2); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(4); + + // ACT + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker accepts because the resource is available. + assertThat(shouldKeep).isTrue(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(2); + + // ACT + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker accepts because the resource is available. + assertThat(shouldKeep).isTrue(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(0); + + // ACT + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker rejects because there are no resources left. + assertThat(shouldKeep).isFalse(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(0); + } + + // Function under test: shouldKeepOperation + // Reason for testing: the local resources should fail to claim, and the existing amount should be + // the same. + // Failure explanation: semaphore claim did not work as expected. + @Test + public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception { + // ARRANGE + configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); + configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); + SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); + resourceSet.resources.put("FOO", new Semaphore(50)); + resourceSet.resources.put("BAR", new Semaphore(100)); + resourceSet.resources.put("BAZ", new Semaphore(200)); + + QueueEntry entry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties( + Platform.Property.newBuilder().setName("resource:FOO").setValue("20")) + .addProperties( + Platform.Property.newBuilder().setName("resource:BAR").setValue("101")) + .addProperties( + Platform.Property.newBuilder().setName("resource:BAZ").setValue("20"))) + .build(); + + // PRE-ASSERT + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(50); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(100); + assertThat(resourceSet.resources.get("BAZ").availablePermits()).isEqualTo(200); + + // ACT + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker rejects because there are no resources left. + // The same amount are returned. + assertThat(shouldKeep).isFalse(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(50); + assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(100); + assertThat(resourceSet.resources.get("BAZ").availablePermits()).isEqualTo(200); + } } diff --git a/src/test/java/build/buildfarm/worker/StubWorkerContext.java b/src/test/java/build/buildfarm/worker/StubWorkerContext.java index d288d67a6..7ccf1182d 100644 --- a/src/test/java/build/buildfarm/worker/StubWorkerContext.java +++ b/src/test/java/build/buildfarm/worker/StubWorkerContext.java @@ -222,4 +222,9 @@ public ResourceLimits commandExecutionSettings(Command command) { public boolean shouldErrorOperationOnRemainingResources() { throw new UnsupportedOperationException(); } + + @Override + public void returnLocalResources(QueueEntry queueEntry) { + throw new UnsupportedOperationException(); + } } diff --git a/src/test/java/build/buildfarm/worker/shard/BUILD b/src/test/java/build/buildfarm/worker/shard/BUILD index b8a7b31ec..c254e41fa 100644 --- a/src/test/java/build/buildfarm/worker/shard/BUILD +++ b/src/test/java/build/buildfarm/worker/shard/BUILD @@ -11,6 +11,7 @@ java_test( "//src/main/java/build/buildfarm/common/config", "//src/main/java/build/buildfarm/instance", "//src/main/java/build/buildfarm/worker", + "//src/main/java/build/buildfarm/worker/resources", "//src/main/java/build/buildfarm/worker/shard", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "//src/test/java/build/buildfarm:test_runner", diff --git a/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java b/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java index dd584c49a..c24e681ab 100644 --- a/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java +++ b/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java @@ -35,6 +35,7 @@ import build.buildfarm.instance.MatchListener; import build.buildfarm.v1test.QueueEntry; import build.buildfarm.worker.WorkerContext; +import build.buildfarm.worker.resources.LocalResourceSet; import com.google.common.collect.ImmutableList; import com.google.protobuf.Duration; import java.util.ArrayList; @@ -105,6 +106,7 @@ WorkerContext createTestContext(Iterable policies) { /* onlyMulticoreTests=*/ false, /* allowBringYourOwnContainer=*/ false, /* errorOperationRemainingResources=*/ false, + /* resourceSet=*/ new LocalResourceSet(), writer); } From 97e3b90c263a27b6e08713d2455a590f7f47405c Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:34:32 -0700 Subject: [PATCH 02/68] build: override grpc dependencies with our dependencies Don't get transitive grpc dependencies, use the ones from our `maven_install(...)` --- defs.bzl | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/defs.bzl b/defs.bzl index ec5185648..869fdd2da 100644 --- a/defs.bzl +++ b/defs.bzl @@ -8,7 +8,7 @@ load( "@io_bazel_rules_docker//repositories:repositories.bzl", container_repositories = "repositories", ) -load("@io_grpc_grpc_java//:repositories.bzl", "grpc_java_repositories") +load("@io_grpc_grpc_java//:repositories.bzl", "grpc_java_repositories", "IO_GRPC_GRPC_JAVA_OVERRIDE_TARGETS") load("@com_google_protobuf//:protobuf_deps.bzl", "protobuf_deps") load("@com_grail_bazel_toolchain//toolchain:rules.bzl", "llvm_toolchain") load("@io_bazel_rules_k8s//k8s:k8s.bzl", "k8s_repositories") @@ -114,6 +114,7 @@ def buildfarm_init(name = "buildfarm"): "org.projectlombok:lombok:1.18.24", ], generate_compat_repositories = True, + override_targets = IO_GRPC_GRPC_JAVA_OVERRIDE_TARGETS, repositories = [ "https://repo1.maven.org/maven2", "https://mirrors.ibiblio.org/pub/mirrors/maven2", From 96f239d5131b17c32d07a46bb583b487472e5d1c Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:35:25 -0700 Subject: [PATCH 03/68] chore(deps): bump protobuf runtime to 3.19.1 --- defs.bzl | 4 ++-- deps.bzl | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/defs.bzl b/defs.bzl index 869fdd2da..ad8d07ced 100644 --- a/defs.bzl +++ b/defs.bzl @@ -79,8 +79,8 @@ def buildfarm_init(name = "buildfarm"): "com.google.guava:guava:32.1.1-jre", "com.google.j2objc:j2objc-annotations:1.1", "com.google.jimfs:jimfs:1.1", - "com.google.protobuf:protobuf-java-util:3.10.0", - "com.google.protobuf:protobuf-java:3.10.0", + "com.google.protobuf:protobuf-java-util:3.19.1", + "com.google.protobuf:protobuf-java:3.19.1", "com.google.truth:truth:0.44", "org.slf4j:slf4j-simple:1.7.35", "com.googlecode.json-simple:json-simple:1.1.1", diff --git a/deps.bzl b/deps.bzl index f802fb27e..e2d2609ac 100644 --- a/deps.bzl +++ b/deps.bzl @@ -36,9 +36,9 @@ def archive_dependencies(third_party): # Needed for "well-known protos" and @com_google_protobuf//:protoc. { "name": "com_google_protobuf", - "sha256": "dd513a79c7d7e45cbaeaf7655289f78fd6b806e52dbbd7018ef4e3cf5cff697a", - "strip_prefix": "protobuf-3.15.8", - "urls": ["https://github.com/protocolbuffers/protobuf/archive/v3.15.8.zip"], + "sha256": "25f1292d4ea6666f460a2a30038eef121e6c3937ae0f61d610611dfb14b0bd32", + "strip_prefix": "protobuf-3.19.1", + "urls": ["https://github.com/protocolbuffers/protobuf/archive/v3.19.1.zip"], }, { "name": "com_github_bazelbuild_buildtools", From af3f34e143c263f10139a9eee6262fb190bdddb9 Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:36:21 -0700 Subject: [PATCH 04/68] chore(deps) add transitive dependencies --- src/main/java/build/buildfarm/server/BUILD | 2 ++ src/main/java/build/buildfarm/worker/shard/BUILD | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/main/java/build/buildfarm/server/BUILD b/src/main/java/build/buildfarm/server/BUILD index eccbf4fda..7244807f8 100644 --- a/src/main/java/build/buildfarm/server/BUILD +++ b/src/main/java/build/buildfarm/server/BUILD @@ -16,6 +16,8 @@ java_library( "//src/main/java/build/buildfarm/metrics/prometheus", "//src/main/java/build/buildfarm/server/services", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "@io_grpc_grpc_proto//:health_proto", + "@io_grpc_grpc_proto//:health_java_proto", "@maven//:com_github_pcj_google_options", "@maven//:com_google_guava_guava", "@maven//:io_grpc_grpc_api", diff --git a/src/main/java/build/buildfarm/worker/shard/BUILD b/src/main/java/build/buildfarm/worker/shard/BUILD index ec48ec5be..4458d6182 100644 --- a/src/main/java/build/buildfarm/worker/shard/BUILD +++ b/src/main/java/build/buildfarm/worker/shard/BUILD @@ -20,6 +20,8 @@ java_library( "//src/main/java/build/buildfarm/worker/resources", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_grpc", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "@io_grpc_grpc_proto//:health_java_proto", + "@io_grpc_grpc_proto//:health_proto", "@googleapis//:google_rpc_error_details_java_proto", "@maven//:com_github_ben_manes_caffeine_caffeine", "@maven//:com_github_pcj_google_options", From 380f8a1bec163adffc7e0c883abf7f5a160a80ac Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:36:33 -0700 Subject: [PATCH 05/68] feat: add Proto reflection service to shard worker To aid connection troubleshooting --- src/main/java/build/buildfarm/worker/shard/Worker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index 7a0d4a60e..038097de7 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -70,6 +70,7 @@ import io.grpc.Status; import io.grpc.Status.Code; import io.grpc.health.v1.HealthCheckResponse.ServingStatus; +import io.grpc.protobuf.services.ProtoReflectionService; import io.grpc.services.HealthStatusManager; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; @@ -158,6 +159,7 @@ private Server createServer( serverBuilder.addService(new ContentAddressableStorageService(instance)); serverBuilder.addService(new ByteStreamService(instance)); serverBuilder.addService(new ShutDownWorkerGracefully(this)); + serverBuilder.addService(ProtoReflectionService.newInstance()); // We will build a worker's server based on it's capabilities. // A worker that is capable of execution will construct an execution pipeline. From 7e7979d037294d1896ed6ae53d8a966e97ffc609 Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:55:18 -0700 Subject: [PATCH 06/68] fixup! build: override grpc dependencies with our dependencies --- defs.bzl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/defs.bzl b/defs.bzl index ad8d07ced..5b0e4f640 100644 --- a/defs.bzl +++ b/defs.bzl @@ -8,7 +8,7 @@ load( "@io_bazel_rules_docker//repositories:repositories.bzl", container_repositories = "repositories", ) -load("@io_grpc_grpc_java//:repositories.bzl", "grpc_java_repositories", "IO_GRPC_GRPC_JAVA_OVERRIDE_TARGETS") +load("@io_grpc_grpc_java//:repositories.bzl", "IO_GRPC_GRPC_JAVA_OVERRIDE_TARGETS", "grpc_java_repositories") load("@com_google_protobuf//:protobuf_deps.bzl", "protobuf_deps") load("@com_grail_bazel_toolchain//toolchain:rules.bzl", "llvm_toolchain") load("@io_bazel_rules_k8s//k8s:k8s.bzl", "k8s_repositories") From 1f9d01fa70f1ecc419c9fca62f5602e5400d66c6 Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:55:29 -0700 Subject: [PATCH 07/68] fixup! chore(deps) add transitive dependencies --- src/main/java/build/buildfarm/server/BUILD | 2 +- src/main/java/build/buildfarm/worker/shard/BUILD | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/build/buildfarm/server/BUILD b/src/main/java/build/buildfarm/server/BUILD index 7244807f8..2cd750a4f 100644 --- a/src/main/java/build/buildfarm/server/BUILD +++ b/src/main/java/build/buildfarm/server/BUILD @@ -16,8 +16,8 @@ java_library( "//src/main/java/build/buildfarm/metrics/prometheus", "//src/main/java/build/buildfarm/server/services", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", - "@io_grpc_grpc_proto//:health_proto", "@io_grpc_grpc_proto//:health_java_proto", + "@io_grpc_grpc_proto//:health_proto", "@maven//:com_github_pcj_google_options", "@maven//:com_google_guava_guava", "@maven//:io_grpc_grpc_api", diff --git a/src/main/java/build/buildfarm/worker/shard/BUILD b/src/main/java/build/buildfarm/worker/shard/BUILD index 4458d6182..b62e4369d 100644 --- a/src/main/java/build/buildfarm/worker/shard/BUILD +++ b/src/main/java/build/buildfarm/worker/shard/BUILD @@ -20,9 +20,9 @@ java_library( "//src/main/java/build/buildfarm/worker/resources", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_grpc", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "@googleapis//:google_rpc_error_details_java_proto", "@io_grpc_grpc_proto//:health_java_proto", "@io_grpc_grpc_proto//:health_proto", - "@googleapis//:google_rpc_error_details_java_proto", "@maven//:com_github_ben_manes_caffeine_caffeine", "@maven//:com_github_pcj_google_options", "@maven//:com_google_code_findbugs_jsr305", From 578589fe6dfe075bbb92eb68b4235ed7a335afb0 Mon Sep 17 00:00:00 2001 From: amishra-u <119983081+amishra-u@users.noreply.github.com> Date: Tue, 24 Oct 2023 11:00:14 -0700 Subject: [PATCH 08/68] Bug: Fix Blocked thread in WriteStreamObserver Caused by CASFile Write (#1486) * Add unit test * Signal Write on complete --- .../build/buildfarm/cas/cfc/CASFileCache.java | 3 + .../buildfarm/cas/cfc/CASFileCacheTest.java | 113 ++++++++++++++++++ 2 files changed, 116 insertions(+) diff --git a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java index aac1b2899..21c90e5fe 100644 --- a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java +++ b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java @@ -878,6 +878,9 @@ public synchronized void reset() { + key.getIdentifier(), e); } finally { + if (closedFuture != null) { + closedFuture.set(null); + } isReset = true; } } diff --git a/src/test/java/build/buildfarm/cas/cfc/CASFileCacheTest.java b/src/test/java/build/buildfarm/cas/cfc/CASFileCacheTest.java index a3d406e23..caaab02b3 100644 --- a/src/test/java/build/buildfarm/cas/cfc/CASFileCacheTest.java +++ b/src/test/java/build/buildfarm/cas/cfc/CASFileCacheTest.java @@ -19,8 +19,11 @@ import static com.google.common.truth.Truth.assertThat; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static com.google.common.util.concurrent.MoreExecutors.shutdownAndAwaitTermination; +import static java.lang.Thread.State.TERMINATED; +import static java.lang.Thread.State.WAITING; import static java.util.concurrent.Executors.newSingleThreadExecutor; import static java.util.concurrent.TimeUnit.MICROSECONDS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.junit.Assert.fail; import static org.mockito.Mockito.any; @@ -58,6 +61,7 @@ import com.google.common.collect.Maps; import com.google.common.jimfs.Configuration; import com.google.common.jimfs.Jimfs; +import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; @@ -77,6 +81,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -1207,6 +1212,114 @@ protected InputStream newExternalInput( assertThat(expected).isNotNull(); } + @Test + public void testConcurrentWrites() throws Exception { + ByteString blob = ByteString.copyFromUtf8("concurrent write"); + Digest digest = DIGEST_UTIL.compute(blob); + UUID uuid = UUID.randomUUID(); + // The same instance of Write will be passed to both the threads, so that the both threads + // try to get same output stream. + Write write = + fileCache.getWrite( + Compressor.Value.IDENTITY, digest, uuid, RequestMetadata.getDefaultInstance()); + + CyclicBarrier barrier = new CyclicBarrier(3); + + Thread write1 = + new Thread( + () -> { + try { + ConcurrentWriteStreamObserver writeStreamObserver = + new ConcurrentWriteStreamObserver(write); + writeStreamObserver.registerCallback(); + barrier.await(); // let both the threads get same write stream. + writeStreamObserver.ownStream(); // let other thread get the ownership of stream + writeStreamObserver.write(blob); + writeStreamObserver.close(); + } catch (Exception e) { + // do nothing + } + }, + "FirstRequest"); + Thread write2 = + new Thread( + () -> { + try { + ConcurrentWriteStreamObserver writeStreamObserver = + new ConcurrentWriteStreamObserver(write); + writeStreamObserver.registerCallback(); + writeStreamObserver.ownStream(); // this thread will get the ownership of stream + barrier.await(); // let both the threads get same write stream. + while (write1.getState() != WAITING) ; // wait for first request to go in wait state + writeStreamObserver.write(blob); + writeStreamObserver.close(); + } catch (Exception e) { + // do nothing + } + }, + "SecondRequest"); + write1.start(); + write2.start(); + barrier.await(); // let both the requests reach the critical section + + // Wait for each write operation to complete, allowing a maximum of 100ms per write. + // Note: A 100ms wait time allowed 1000 * 8 successful test runs. + // In certain scenario, even this wait time may not be enough and test still be called flaky. + // But setting wait time 0 may cause test to wait forever (if there is issue in code) and the + // build might fail with timeout error. + write1.join(100); + write2.join(100); + + assertThat(write1.getState()).isEqualTo(TERMINATED); + assertThat(write2.getState()).isEqualTo(TERMINATED); + } + + static class ConcurrentWriteStreamObserver { + Write write; + FeedbackOutputStream out; + + ConcurrentWriteStreamObserver(Write write) { + this.write = write; + } + + void registerCallback() { + Futures.addCallback( + write.getFuture(), + new FutureCallback() { + @Override + public void onSuccess(Long committedSize) { + commit(); + } + + @Override + public void onFailure(Throwable t) { + // do nothing + } + }, + directExecutor()); + } + + synchronized void ownStream() throws Exception { + this.out = write.getOutput(10, MILLISECONDS, () -> {}); + } + /** + * Request 1 may invoke this method for request 2 or vice-versa via callback on + * write.getFuture(). Synchronization is necessary to prevent conflicts when this method is + * called simultaneously by different threads. + */ + synchronized void commit() { + // critical section + } + + void write(ByteString data) throws IOException { + data.writeTo(out); + } + + void close() throws IOException { + out.close(); + } + } + @RunWith(JUnit4.class) public static class NativeFileDirsIndexInMemoryCASFileCacheTest extends CASFileCacheTest { public NativeFileDirsIndexInMemoryCASFileCacheTest() throws IOException { From dfa5937090d36e7d39fae577c978412367b71c6c Mon Sep 17 00:00:00 2001 From: Stefano Baghino Date: Tue, 24 Oct 2023 21:38:09 +0200 Subject: [PATCH 09/68] Pin the Java toolchain to `remotejdk_17` (#1509) Closes #1508 Cleanups: - remove the unused `ubuntu-bionic` base image - replace `ubuntu-jammy:jammy-java11-gcc` with `ubuntu-mantic:mantic-java17-gcc` - replace `amazoncorretto:19` with `ubuntu-mantic:mantic-java17-gcc` - swap inverted log file names in a file --- .bazelci/run_server_test.sh | 4 ++-- .bazelrc | 6 +++++ BUILD | 6 ++--- images.bzl | 22 +++---------------- jvm_flags.bzl | 10 +++++++-- .../common/processes/JavaProcessWrapper.java | 6 ++++- .../bazel/processes/PersistentWorkerTest.java | 3 ++- src/main/java/build/buildfarm/BUILD | 6 ++--- src/test/java/build/buildfarm/cas/BUILD | 4 ++-- src/test/java/build/buildfarm/common/BUILD | 4 ++-- 10 files changed, 36 insertions(+), 35 deletions(-) diff --git a/.bazelci/run_server_test.sh b/.bazelci/run_server_test.sh index 21b6d7038..54e4658b1 100755 --- a/.bazelci/run_server_test.sh +++ b/.bazelci/run_server_test.sh @@ -8,11 +8,11 @@ bazel build //src/main/java/build/buildfarm:buildfarm-shard-worker bazel build //src/main/java/build/buildfarm:buildfarm-server # Start a single worker -bazel run //src/main/java/build/buildfarm:buildfarm-shard-worker $(pwd)/examples/config.minimal.yml > server.log 2>&1 & +bazel run //src/main/java/build/buildfarm:buildfarm-shard-worker $(pwd)/examples/config.minimal.yml > worker.log 2>&1 & echo "Started buildfarm-shard-worker..." # Start a single server -bazel run //src/main/java/build/buildfarm:buildfarm-server $(pwd)/examples/config.minimal.yml > worker.log 2>&1 & +bazel run //src/main/java/build/buildfarm:buildfarm-server $(pwd)/examples/config.minimal.yml > server.log 2>&1 & echo "Started buildfarm-server..." echo "Wait for startup to finish..." diff --git a/.bazelrc b/.bazelrc index adbb3c68f..c3caf4d8f 100644 --- a/.bazelrc +++ b/.bazelrc @@ -1,3 +1,9 @@ +build --java_language_version=17 +build --java_runtime_version=remotejdk_17 + +build --tool_java_language_version=17 +build --tool_java_runtime_version=remotejdk_17 + common --enable_platform_specific_config build:fuse --define=fuse=true diff --git a/BUILD b/BUILD index f552ea9d7..eb773e1a1 100644 --- a/BUILD +++ b/BUILD @@ -120,7 +120,7 @@ sh_binary( java_image( name = "buildfarm-server", args = ["/app/build_buildfarm/examples/config.minimal.yml"], - base = "@amazon_corretto_java_image_base//image", + base = "@ubuntu-mantic//image", classpath_resources = [ "//src/main/java/build/buildfarm:configs", ], @@ -148,14 +148,14 @@ oss_audit( # Download cgroup-tools so that the worker is able to restrict actions via control groups. download_pkgs( name = "worker_pkgs", - image_tar = "@ubuntu-jammy//image", + image_tar = "@ubuntu-mantic//image", packages = ["cgroup-tools"], tags = ["container"], ) install_pkgs( name = "worker_pkgs_image", - image_tar = "@ubuntu-jammy//image", + image_tar = "@ubuntu-mantic//image", installables_tar = ":worker_pkgs.tar", installation_cleanup_commands = "rm -rf /var/lib/apt/lists/*", output_image_name = "worker_pkgs_image", diff --git a/images.bzl b/images.bzl index 9ab0a8b0b..939a752ed 100644 --- a/images.bzl +++ b/images.bzl @@ -27,25 +27,9 @@ def buildfarm_images(): ) container_pull( - name = "ubuntu-bionic", - digest = "sha256:4bc527c7a288da405f2041928c63d0a6479a120ad63461c2f124c944def54be2", + name = "ubuntu-mantic", + digest = "sha256:1419bba15470a95242e917b3abcd8981ae36707b99df5ac705e1eee4d920c51c", registry = "index.docker.io", repository = "bazelbuild/buildfarm-worker-base", - tag = "bionic-java11-gcc", - ) - - container_pull( - name = "ubuntu-jammy", - digest = "sha256:da847ee259ebe7f00631a2f0146d9add60ff0f94b031a2e522ce94c78b1335c2", - registry = "index.docker.io", - repository = "bazelbuild/buildfarm-worker-base", - tag = "jammy-java11-gcc", - ) - - container_pull( - name = "amazon_corretto_java_image_base", - registry = "index.docker.io", - repository = "amazoncorretto", - tag = "19", - digest = "sha256:81d0df4412140416b27211c999e1f3c4565ae89a5cd92889475d20af422ba507", + tag = "mantic-java17-gcc", ) diff --git a/jvm_flags.bzl b/jvm_flags.bzl index 363f16146..440e0718a 100644 --- a/jvm_flags.bzl +++ b/jvm_flags.bzl @@ -54,6 +54,12 @@ def ensure_accurate_metadata(): "//config:windows": ["-Dsun.nio.fs.ensureAccurateMetadata=true"], }) +def add_opens_sun_nio_fs(): + return select({ + "//conditions:default": [], + "//config:windows": ["--add-opens java.base/sun.nio.fs=ALL-UNNAMED"], + }) + def server_telemetry(): return select({ "//config:open_telemetry": SERVER_TELEMETRY_JVM_FLAGS, @@ -67,7 +73,7 @@ def worker_telemetry(): }) def server_jvm_flags(): - return RECOMMENDED_JVM_FLAGS + DEFAULT_LOGGING_CONFIG + ensure_accurate_metadata() + server_telemetry() + return RECOMMENDED_JVM_FLAGS + DEFAULT_LOGGING_CONFIG + ensure_accurate_metadata() + add_opens_sun_nio_fs() + server_telemetry() def worker_jvm_flags(): - return RECOMMENDED_JVM_FLAGS + DEFAULT_LOGGING_CONFIG + ensure_accurate_metadata() + worker_telemetry() + return RECOMMENDED_JVM_FLAGS + DEFAULT_LOGGING_CONFIG + ensure_accurate_metadata() + add_opens_sun_nio_fs() + worker_telemetry() diff --git a/persistentworkers/src/main/java/persistent/common/processes/JavaProcessWrapper.java b/persistentworkers/src/main/java/persistent/common/processes/JavaProcessWrapper.java index a27b6a9e9..89f2e6a5b 100644 --- a/persistentworkers/src/main/java/persistent/common/processes/JavaProcessWrapper.java +++ b/persistentworkers/src/main/java/persistent/common/processes/JavaProcessWrapper.java @@ -10,12 +10,16 @@ public class JavaProcessWrapper extends ProcessWrapper { + // Get the path of the JVM from the current process to avoid breaking the Bazel sandbox + public static final String CURRENT_JVM_COMMAND = + ProcessHandle.current().info().command().orElseThrow(() -> new RuntimeException("Unable to retrieve the path of the running JVM")); + public JavaProcessWrapper( Path workDir, String classPath, String fullClassName, String[] args ) throws IOException { super(workDir, cmdArgs( new String[]{ - "java", + CURRENT_JVM_COMMAND, "-cp", classPath, fullClassName diff --git a/persistentworkers/src/test/java/persistent/bazel/processes/PersistentWorkerTest.java b/persistentworkers/src/test/java/persistent/bazel/processes/PersistentWorkerTest.java index 0cdc68a7f..971239420 100644 --- a/persistentworkers/src/test/java/persistent/bazel/processes/PersistentWorkerTest.java +++ b/persistentworkers/src/test/java/persistent/bazel/processes/PersistentWorkerTest.java @@ -16,6 +16,7 @@ import persistent.bazel.client.PersistentWorker; import persistent.bazel.client.WorkerKey; +import persistent.common.processes.JavaProcessWrapper; import persistent.testutil.ProcessUtils; import persistent.testutil.WorkerUtils; @@ -55,7 +56,7 @@ public void endToEndAdder() throws Exception { ); ImmutableList initCmd = ImmutableList.of( - "java", + JavaProcessWrapper.CURRENT_JVM_COMMAND, "-cp", jarPath.toString(), "adder.Adder", diff --git a/src/main/java/build/buildfarm/BUILD b/src/main/java/build/buildfarm/BUILD index 601aa38eb..3cbdeb523 100644 --- a/src/main/java/build/buildfarm/BUILD +++ b/src/main/java/build/buildfarm/BUILD @@ -1,4 +1,4 @@ -load("//:jvm_flags.bzl", "ensure_accurate_metadata") +load("//:jvm_flags.bzl", "add_opens_sun_nio_fs", "ensure_accurate_metadata") package( default_visibility = ["//src:__subpackages__"], @@ -15,7 +15,7 @@ java_binary( classpath_resources = [ ":configs", ], - jvm_flags = ensure_accurate_metadata(), + jvm_flags = ensure_accurate_metadata() + add_opens_sun_nio_fs(), main_class = "build.buildfarm.server.BuildFarmServer", visibility = ["//visibility:public"], runtime_deps = [ @@ -29,7 +29,7 @@ java_binary( classpath_resources = [ ":configs", ], - jvm_flags = ensure_accurate_metadata(), + jvm_flags = ensure_accurate_metadata() + add_opens_sun_nio_fs(), main_class = "build.buildfarm.worker.shard.Worker", visibility = ["//visibility:public"], runtime_deps = [ diff --git a/src/test/java/build/buildfarm/cas/BUILD b/src/test/java/build/buildfarm/cas/BUILD index ff7774e06..fc127b1ea 100644 --- a/src/test/java/build/buildfarm/cas/BUILD +++ b/src/test/java/build/buildfarm/cas/BUILD @@ -1,10 +1,10 @@ -load("//:jvm_flags.bzl", "ensure_accurate_metadata") +load("//:jvm_flags.bzl", "add_opens_sun_nio_fs", "ensure_accurate_metadata") java_test( name = "tests", size = "small", srcs = glob(["**/*.java"]), - jvm_flags = ensure_accurate_metadata(), + jvm_flags = ensure_accurate_metadata() + add_opens_sun_nio_fs(), test_class = "build.buildfarm.AllTests", deps = [ "//src/main/java/build/buildfarm/cas", diff --git a/src/test/java/build/buildfarm/common/BUILD b/src/test/java/build/buildfarm/common/BUILD index bda4f4243..70ff4abc9 100644 --- a/src/test/java/build/buildfarm/common/BUILD +++ b/src/test/java/build/buildfarm/common/BUILD @@ -1,10 +1,10 @@ -load("//:jvm_flags.bzl", "ensure_accurate_metadata") +load("//:jvm_flags.bzl", "add_opens_sun_nio_fs", "ensure_accurate_metadata") java_test( name = "tests", size = "small", srcs = glob(["*.java"]), - jvm_flags = ensure_accurate_metadata(), + jvm_flags = ensure_accurate_metadata() + add_opens_sun_nio_fs(), test_class = "build.buildfarm.AllTests", deps = [ "//src/main/java/build/buildfarm/common", From f6459d199d1a40bdd8b31a75d7c19da7a1af69f4 Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Wed, 18 Oct 2023 13:32:18 -0700 Subject: [PATCH 10/68] docs: add markdown language specifiers for code blocks --- README.md | 28 ++++++++--------- _site/docs/configuration/configuration.md | 32 ++++++++++---------- _site/docs/execution/execution_policies.md | 26 ++++++++++------ _site/docs/execution/execution_properties.md | 25 +++++++++------ _site/docs/quick_start.md | 25 +++++++++------ 5 files changed, 77 insertions(+), 59 deletions(-) diff --git a/README.md b/README.md index 675fe94db..cd9caf8ee 100644 --- a/README.md +++ b/README.md @@ -19,8 +19,8 @@ All commandline options override corresponding config settings. Run via -``` -docker run -d --rm --name buildfarm-redis -p 6379:6379 redis:5.0.9 +```shell +$ docker run -d --rm --name buildfarm-redis -p 6379:6379 redis:5.0.9 redis-cli config set stop-writes-on-bgsave-error no ``` @@ -28,8 +28,8 @@ redis-cli config set stop-writes-on-bgsave-error no Run via -``` -bazelisk run //src/main/java/build/buildfarm:buildfarm-server -- +```shell +$ bazelisk run //src/main/java/build/buildfarm:buildfarm-server -- Ex: bazelisk run //src/main/java/build/buildfarm:buildfarm-server -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml ``` @@ -40,8 +40,8 @@ Ex: bazelisk run //src/main/java/build/buildfarm:buildfarm-server -- --jvm_flag= Run via -``` -bazelisk run //src/main/java/build/buildfarm:buildfarm-shard-worker -- +```shell +$ bazelisk run //src/main/java/build/buildfarm:buildfarm-shard-worker -- Ex: bazelisk run //src/main/java/build/buildfarm:buildfarm-shard-worker -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml @@ -53,9 +53,9 @@ Ex: bazelisk run //src/main/java/build/buildfarm:buildfarm-shard-worker -- --jvm To use the example configured buildfarm with bazel (version 1.0 or higher), you can configure your `.bazelrc` as follows: -``` +```shell $ cat .bazelrc -build --remote_executor=grpc://localhost:8980 +$ build --remote_executor=grpc://localhost:8980 ``` Then run your build as you would normally do. @@ -67,20 +67,20 @@ Buildfarm uses [Java's Logging framework](https://docs.oracle.com/javase/10/core You can use typical Java logging configuration to filter these results and observe the flow of executions through your running services. An example `logging.properties` file has been provided at [examples/logging.properties](examples/logging.properties) for use as follows: -``` -bazel run //src/main/java/build/buildfarm:buildfarm-server -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml +```shell +$ bazel run //src/main/java/build/buildfarm:buildfarm-server -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml ``` and -``` -bazel run //src/main/java/build/buildfarm:buildfarm-shard-worker -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml +``` shell +$ bazel run //src/main/java/build/buildfarm:buildfarm-shard-worker -- --jvm_flag=-Djava.util.logging.config.file=$PWD/examples/logging.properties $PWD/examples/config.minimal.yml ``` To attach a remote debugger, run the executable with the `--debug=` flag. For example: -``` -bazel run //src/main/java/build/buildfarm:buildfarm-server -- --debug=5005 $PWD/examples/config.minimal.yml +```shell +$ bazel run //src/main/java/build/buildfarm:buildfarm-server -- --debug=5005 $PWD/examples/config.minimal.yml ``` diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index ae913adf8..ab3638f5b 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -7,7 +7,7 @@ has_children: true Minimal required: -``` +```yaml backplane: redisUri: "redis://localhost:6379" queues: @@ -38,7 +38,7 @@ For an example configuration containing all of the configuration values, see `ex Example: -``` +```yaml digestFunction: SHA1 defaultActionTimeout: 1800 maximumActionTimeout: 1800 @@ -79,7 +79,7 @@ worker: Example: -``` +```yaml server: instanceType: SHARD name: shard @@ -96,7 +96,7 @@ server: Example: -``` +```yaml server: grpcMetrics: enabled: false @@ -114,7 +114,7 @@ server: Example: -``` +```yaml server: caches: directoryCacheMaxEntries: 10000 @@ -132,7 +132,7 @@ server: Example: -``` +```yaml server: admin: deploymentEnvironment: AWS @@ -151,14 +151,14 @@ server: Example: -``` +```yaml server: metrics: publisher: log logLevel: INFO ``` -``` +```yaml server: metrics: publisher: aws @@ -207,7 +207,7 @@ server: Example: -``` +```yaml backplane: type: SHARD redisUri: "redis://localhost:6379" @@ -224,7 +224,7 @@ backplane: Example: -``` +```yaml backplane: type: SHARD redisUri: "redis://localhost:6379" @@ -262,7 +262,7 @@ backplane: | realInputDirectories | List of Strings, _external_ | | A list of paths that will not be subject to the effects of linkInputDirectories setting, may also be used to provide writable directories as input roots for actions which expect to be able to write to an input location and will fail if they cannot | | gracefulShutdownSeconds | Integer, 0 | | Time in seconds to allow for operations in flight to finish when shutdown signal is received | -``` +```yaml worker: port: 8981 publicName: "localhost:8981" @@ -279,7 +279,7 @@ worker: Example: -``` +```yaml worker: capabilities: cas: true @@ -296,7 +296,7 @@ worker: Example: -``` +```yaml worker: sandboxSettings: alwaysUse: true @@ -313,7 +313,7 @@ worker: Example: -``` +```yaml worker: dequeueMatchSettings: acceptEverything: true @@ -333,7 +333,7 @@ worker: Example: -``` +```yaml worker: storages: - type: FILESYSTEM @@ -361,7 +361,7 @@ worker: Example: -``` +```yaml worker: executionPolicies: - name: test diff --git a/_site/docs/execution/execution_policies.md b/_site/docs/execution/execution_policies.md index 19698a992..f0eaf295d 100644 --- a/_site/docs/execution/execution_policies.md +++ b/_site/docs/execution/execution_policies.md @@ -17,7 +17,7 @@ This policy type specifies that a worker should prepend a single path, and a num This example will use the buildfarm-provided executable `as-nobody`, which will upon execution demote itself to a `nobody` effective process owner uid, and perform an `execvp(2)` with the remaining provided program arguments, which will subsequently execute as a user that no longer matches the worker process. -``` +```yaml # default wrapper policy application worker: executionPolicies: @@ -50,7 +50,8 @@ These wrappers are used for detecting actions that rely on time. Below is a dem This addresses two problems in regards to an action's dependence on time. The 1st problem is when an action takes longer than it should because it's sleeping unnecessarily. The 2nd problem is when an action relies on time which causes it to eventually be broken on master despite the code not changing. Both problems are expressed below as unit tests. We demonstrate a time-spoofing mechanism (the re-writing of syscalls) which allows us to detect these problems generically over any action. The objective is to analyze builds for performance inefficiency and discover future instabilities before they occur. ### Issue 1 (slow test) -``` + +```bash #!/bin/bash set -euo pipefail @@ -58,16 +59,19 @@ echo -n "testing... " sleep 10; echo "done" ``` + The test takes 10 seconds to run on average. -``` -bazel test --runs_per_test=10 --config=remote //cloud/buildfarm:sleep_test + +```shell +$ bazel test --runs_per_test=10 --config=remote //cloud/buildfarm:sleep_test //cloud/buildfarm:sleep_test PASSED in 10.2s Stats over 10 runs: max = 10.2s, min = 10.1s, avg = 10.2s, dev = 0.0s ``` We can check for performance improvements by using the `skip-sleep` option. -``` -bazel test --runs_per_test=10 --config=remote --remote_default_exec_properties='skip-sleep=true' //cloud/buildfarm:sleep_test + +```shell +$ bazel test --runs_per_test=10 --config=remote --remote_default_exec_properties='skip-sleep=true' //cloud/buildfarm:sleep_test //cloud/buildfarm:sleep_test PASSED in 1.0s Stats over 10 runs: max = 1.0s, min = 0.9s, avg = 1.0s, dev = 0.0s ``` @@ -75,7 +79,8 @@ bazel test --runs_per_test=10 --config=remote --remote_default_exec_properties=' Now the test is 10x faster. If skipping sleep makes an action perform significantly faster without affecting its success rate, that would warrant further investigation into the action's implementation. ### Issue 2 (future failing test) -``` + +```bash #!/bin/bash set -euo pipefail @@ -89,12 +94,15 @@ echo "Times change." date exit -1; ``` + The test passes today, but will it pass tomorrow? Will it pass a year from now? We can find out by using the `time-shift` option. -``` -bazel test --test_output=streamed --remote_default_exec_properties='time-shift=31556952' --config=remote //cloud/buildfarm:future_fail + +```shell +$ bazel test --test_output=streamed --remote_default_exec_properties='time-shift=31556952' --config=remote //cloud/buildfarm:future_fail INFO: Found 1 test target... Times change. Mon Sep 25 18:31:09 UTC 2023 //cloud/buildfarm:future_fail FAILED in 18.0s ``` + Time is shifted to the year 2023 and the test now fails. We can fix the problem before others see it. diff --git a/_site/docs/execution/execution_properties.md b/_site/docs/execution/execution_properties.md index 85579ed09..7966c70dd 100644 --- a/_site/docs/execution/execution_properties.md +++ b/_site/docs/execution/execution_properties.md @@ -76,37 +76,42 @@ Despite being given 1 core, they see all of the cpus and decide to spawn that ma **Standard Example:** This test will succeed when env var TESTVAR is foobar, and fail otherwise. -``` + +```shell #!/bin/bash [ "$TESTVAR" = "foobar" ] ``` -``` -./bazel test \ + +```shell +$ ./bazel test \ --remote_executor=grpc://127.0.0.1:8980 --noremote_accept_cached --nocache_test_results \ //env_test:main FAIL ``` -``` -./bazel test --remote_default_exec_properties='env-vars={"TESTVAR": "foobar"}' \ +```shell +$ ./bazel test --remote_default_exec_properties='env-vars={"TESTVAR": "foobar"}' \ --remote_executor=grpc://127.0.0.1:8980 --noremote_accept_cached --nocache_test_results \ //env_test:main PASS ``` + **Template Example:** If you give a range of cores, buildfarm has the authority to decide how many your operation actually claims. You can let buildfarm resolve this value for you (via [mustache](https://mustache.github.io/)). -``` +```bash #!/bin/bash [ "$MKL_NUM_THREADS" = "1" ] ``` -``` -./bazel test \ + +```shell +$ ./bazel test \ --remote_executor=grpc://127.0.0.1:8980 --noremote_accept_cached --nocache_test_results \ //env_test:main FAIL ``` -``` -./bazel test \ + +```shell +$ ./bazel test \ --remote_default_exec_properties='env-vars="MKL_NUM_THREADS": "{{limits.cpu.claimed}}"' \ --remote_executor=grpc://127.0.0.1:8980 --noremote_accept_cached --nocache_test_results \ //env_test:main diff --git a/_site/docs/quick_start.md b/_site/docs/quick_start.md index 7af957ee6..8a9a9234d 100644 --- a/_site/docs/quick_start.md +++ b/_site/docs/quick_start.md @@ -25,7 +25,8 @@ Let's start with a bazel workspace with a single file to compile into an executa Create a new directory for our workspace and add the following files: `main.cc`: -``` + +```c #include int main( int argc, char *argv[] ) @@ -35,7 +36,8 @@ int main( int argc, char *argv[] ) ``` `BUILD`: -``` + +```starlark cc_binary( name = "main", srcs = ["main.cc"], @@ -118,15 +120,18 @@ That `2 remote` indicates that your compile and link ran remotely. Congratulatio ## Container Quick Start To bring up a minimal buildfarm cluster, you can run: + +```shell +$ ./examples/bf-run start ``` -./examples/bf-run start -``` + This will start all of the necessary containers at the latest version. Once the containers are up, you can build with `bazel run --remote_executor=grpc://localhost:8980 :main`. To stop the containers, run: -``` -./examples/bf-run stop + +```shell +$ ./examples/bf-run stop ``` ## Next Steps @@ -137,8 +142,8 @@ We've started our worker on the same host as our server, and also the same host You can now easily launch a new Buildfarm cluster locally or in AWS using an open sourced [Buildfarm Manager](https://github.com/80degreeswest/bfmgr). -``` -wget https://github.com/80degreeswest/bfmgr/releases/download/1.0.7/bfmgr-1.0.7.jar -java -jar bfmgr-1.0.7.jar -Navigate to http://localhost +```shell +$ wget https://github.com/80degreeswest/bfmgr/releases/download/1.0.7/bfmgr-1.0.7.jar +$ java -jar bfmgr-1.0.7.jar +$ open http://localhost ``` From 018e177642caebf76f8baeeba2b15c7c3b6f5286 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 25 Oct 2023 16:54:59 -0400 Subject: [PATCH 11/68] Support OutputPaths in OutputDirectory Specifying any number of OutputPaths will ignore OutputFiles (consistent with uploads). Where an OutputPath specifies an output directory, the action must be able to create the directory itself. --- .../worker/shard/CFCExecFileSystem.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java index 041868f69..d042ee0fb 100644 --- a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java +++ b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java @@ -397,16 +397,25 @@ private Set linkedDirectories( return ImmutableSet.of(); } + private OutputDirectory createOutputDirectory(Command command) { + Iterable files; + Iterable dirs; + if (command.getOutputPathsCount() != 0) { + files = command.getOutputPathsList(); + dirs = ImmutableList.of(); // output paths require the action to create their own directory + } else { + files = command.getOutputFilesList(); + dirs = command.getOutputDirectoriesList(); + } + return OutputDirectory.parse(files, dirs, command.getEnvironmentVariablesList()); + } + @Override public Path createExecDir( String operationName, Map directoriesIndex, Action action, Command command) throws IOException, InterruptedException { Digest inputRootDigest = action.getInputRootDigest(); - OutputDirectory outputDirectory = - OutputDirectory.parse( - command.getOutputFilesList(), - command.getOutputDirectoriesList(), - command.getEnvironmentVariablesList()); + OutputDirectory outputDirectory = createOutputDirectory(command); Path execDir = root.resolve(operationName); if (Files.exists(execDir)) { From 8b370138da8f21401e78ed3d931e50194b1f4a06 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 26 Oct 2023 11:50:30 -0400 Subject: [PATCH 12/68] Permit Absolute Symlink Targets with configuration Partial specification of the absolute symlink response per REAPI. Remaining work will be in output identification. --- _site/docs/configuration/configuration.md | 15 +++--- examples/config.yml | 1 + .../common/config/BuildfarmConfigs.java | 1 + .../server/AbstractServerInstance.java | 18 +++++++ .../instance/shard/ShardInstance.java | 14 +++++ .../worker/shard/CFCExecFileSystem.java | 7 ++- .../build/buildfarm/worker/shard/Worker.java | 1 + .../server/AbstractServerInstanceTest.java | 51 +++++++++++++++++++ 8 files changed, 100 insertions(+), 8 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index ab3638f5b..ccfc8e026 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -28,13 +28,14 @@ For an example configuration containing all of the configuration values, see `ex ### Common -| Configuration | Accepted and _Default_ Values | Command Line Argument | Description | -|----------------------|-------------------------------|-----------------------|---------------------------------------------------| -| digestFunction | _SHA256_, SHA1 | | Digest function for this implementation | -| defaultActionTimeout | Integer, _600_ | | Default timeout value for an action (seconds) | -| maximumActionTimeout | Integer, _3600_ | | Maximum allowed action timeout (seconds) | -| maxEntrySizeBytes | Long, _2147483648_ | | Maximum size of a single blob accepted (bytes) | -| prometheusPort | Integer, _9090_ | --prometheus_port | Listening port of the Prometheus metrics endpoint | +| Configuration | Accepted and _Default_ Values | Command Line Argument | Description | +|------------------------------|-------------------------------|-----------------------|--------------------------------------------------------------| +| digestFunction | _SHA256_, SHA1 | | Digest function for this implementation | +| defaultActionTimeout | Integer, _600_ | | Default timeout value for an action (seconds) | +| maximumActionTimeout | Integer, _3600_ | | Maximum allowed action timeout (seconds) | +| maxEntrySizeBytes | Long, _2147483648_ | | Maximum size of a single blob accepted (bytes) | +| prometheusPort | Integer, _9090_ | --prometheus_port | Listening port of the Prometheus metrics endpoint | +| allowSymlinkTargetAbsolute | boolean, _false_ | | Permit inputs to contain symlinks with absolute path targets | Example: diff --git a/examples/config.yml b/examples/config.yml index bce19a3d6..5229fcd07 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -3,6 +3,7 @@ defaultActionTimeout: 600 maximumActionTimeout: 3600 maxEntrySizeBytes: 2147483648 # 2 * 1024 * 1024 * 1024 prometheusPort: 9090 +allowSymlinkTargetAbsolute: false server: instanceType: SHARD name: shard diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java index 444d25a13..d6ebbe9e1 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java @@ -36,6 +36,7 @@ public final class BuildfarmConfigs { private long maximumActionTimeout = 3600; private long maxEntrySizeBytes = 2147483648L; // 2 * 1024 * 1024 * 1024 private int prometheusPort = 9090; + private boolean allowSymlinkTargetAbsolute = false; private Server server = new Server(); private Backplane backplane = new Backplane(); private Worker worker = new Worker(); diff --git a/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java b/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java index d828b7bd8..541e13701 100644 --- a/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java +++ b/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java @@ -178,6 +178,8 @@ public abstract class AbstractServerInstance implements Instance { public static final String ENVIRONMENT_VARIABLES_NOT_SORTED = "The `Command`'s `environment_variables` are not correctly sorted by `name`."; + public static final String SYMLINK_TARGET_ABSOLUTE = "A symlink target is absolute."; + public static final String MISSING_ACTION = "The action was not found in the CAS."; public static final String MISSING_COMMAND = "The command was not found in the CAS."; @@ -790,6 +792,7 @@ public static void validateActionInputDirectory( Stack pathDigests, Set visited, Map directoriesIndex, + boolean allowSymlinkTargetAbsolute, Consumer onInputFile, Consumer onInputDirectory, Consumer onInputDigest, @@ -838,6 +841,14 @@ public static void validateActionInputDirectory( .setSubject("/" + directoryPath + ": " + lastSymlinkName + " > " + symlinkName) .setDescription(DIRECTORY_NOT_SORTED); } + String symlinkTarget = symlinkNode.getTarget(); + if (!allowSymlinkTargetAbsolute && symlinkTarget.charAt(0) == '/') { + preconditionFailure + .addViolationsBuilder() + .setType(VIOLATION_TYPE_INVALID) + .setSubject("/" + directoryPath + ": " + symlinkName + " -> " + symlinkTarget) + .setDescription(SYMLINK_TARGET_ABSOLUTE); + } /* FIXME serverside validity check? regex? Preconditions.checkState( isValidFilename(symlinkName), @@ -907,6 +918,7 @@ public static void validateActionInputDirectory( pathDigests, visited, directoriesIndex, + allowSymlinkTargetAbsolute, onInputFile, onInputDirectory, onInputDigest, @@ -922,6 +934,7 @@ private static void validateActionInputDirectoryDigest( Stack pathDigests, Set visited, Map directoriesIndex, + boolean allowSymlinkTargetAbsolute, Consumer onInputFile, Consumer onInputDirectory, Consumer onInputDigest, @@ -946,6 +959,7 @@ private static void validateActionInputDirectoryDigest( pathDigests, visited, directoriesIndex, + allowSymlinkTargetAbsolute, onInputFile, onInputDirectory, onInputDigest, @@ -1203,12 +1217,16 @@ protected void validateAction( ImmutableSet.Builder inputFilesBuilder = ImmutableSet.builder(); inputDirectoriesBuilder.add(ACTION_INPUT_ROOT_DIRECTORY_PATH); + boolean allowSymlinkTargetAbsolute = + getCacheCapabilities().getSymlinkAbsolutePathStrategy() + == SymlinkAbsolutePathStrategy.Value.ALLOWED; validateActionInputDirectoryDigest( ACTION_INPUT_ROOT_DIRECTORY_PATH, action.getInputRootDigest(), new Stack<>(), new HashSet<>(), directoriesIndex, + allowSymlinkTargetAbsolute, inputFilesBuilder::add, inputDirectoriesBuilder::add, onInputDigest, diff --git a/src/main/java/build/buildfarm/instance/shard/ShardInstance.java b/src/main/java/build/buildfarm/instance/shard/ShardInstance.java index c1c07b08f..241f1a8fa 100644 --- a/src/main/java/build/buildfarm/instance/shard/ShardInstance.java +++ b/src/main/java/build/buildfarm/instance/shard/ShardInstance.java @@ -47,6 +47,7 @@ import build.bazel.remote.execution.v2.Action; import build.bazel.remote.execution.v2.ActionResult; import build.bazel.remote.execution.v2.BatchReadBlobsResponse.Response; +import build.bazel.remote.execution.v2.CacheCapabilities; import build.bazel.remote.execution.v2.Command; import build.bazel.remote.execution.v2.Compressor; import build.bazel.remote.execution.v2.Digest; @@ -60,6 +61,7 @@ import build.bazel.remote.execution.v2.Platform.Property; import build.bazel.remote.execution.v2.RequestMetadata; import build.bazel.remote.execution.v2.ResultsCachePolicy; +import build.bazel.remote.execution.v2.SymlinkAbsolutePathStrategy; import build.buildfarm.actioncache.ActionCache; import build.buildfarm.actioncache.ShardActionCache; import build.buildfarm.backplane.Backplane; @@ -2726,4 +2728,16 @@ private boolean inDenyList(RequestMetadata requestMetadata) throws IOException { } return backplane.isBlacklisted(requestMetadata); } + + @Override + protected CacheCapabilities getCacheCapabilities() { + SymlinkAbsolutePathStrategy.Value symlinkAbsolutePathStrategy = + configs.isAllowSymlinkTargetAbsolute() + ? SymlinkAbsolutePathStrategy.Value.ALLOWED + : SymlinkAbsolutePathStrategy.Value.DISALLOWED; + return super.getCacheCapabilities() + .toBuilder() + .setSymlinkAbsolutePathStrategy(symlinkAbsolutePathStrategy) + .build(); + } } diff --git a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java index d042ee0fb..dba809fdc 100644 --- a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java +++ b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java @@ -82,6 +82,9 @@ class CFCExecFileSystem implements ExecFileSystem { // indicate symlinking above for a set of matching paths private final Iterable linkedInputDirectories; + // permit symlinks to point to absolute paths in inputs + private final boolean allowSymlinkTargetAbsolute; + private final Map> rootInputFiles = new ConcurrentHashMap<>(); private final Map> rootInputDirectories = new ConcurrentHashMap<>(); private final ExecutorService fetchService = BuildfarmExecutors.getFetchServicePool(); @@ -95,6 +98,7 @@ class CFCExecFileSystem implements ExecFileSystem { @Nullable UserPrincipal owner, boolean linkInputDirectories, Iterable linkedInputDirectories, + boolean allowSymlinkTargetAbsolute, ExecutorService removeDirectoryService, ExecutorService accessRecorder) { this.root = root; @@ -104,6 +108,7 @@ class CFCExecFileSystem implements ExecFileSystem { this.linkedInputDirectories = Iterables.transform( linkedInputDirectories, realInputDirectory -> Pattern.compile(realInputDirectory)); + this.allowSymlinkTargetAbsolute = allowSymlinkTargetAbsolute; this.removeDirectoryService = removeDirectoryService; this.accessRecorder = accessRecorder; } @@ -179,7 +184,7 @@ public InputStream newInput(Compressor.Value compressor, Digest digest, long off private ListenableFuture putSymlink(Path path, SymlinkNode symlinkNode) { Path symlinkPath = path.resolve(symlinkNode.getName()); Path relativeTargetPath = path.getFileSystem().getPath(symlinkNode.getTarget()); - checkState(!relativeTargetPath.isAbsolute()); + checkState(allowSymlinkTargetAbsolute || !relativeTargetPath.isAbsolute()); return listeningDecorator(fetchService) .submit( () -> { diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index 038097de7..eaf8b6cb1 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -337,6 +337,7 @@ private ExecFileSystem createCFCExecFileSystem( owner, configs.getWorker().isLinkInputDirectories(), configs.getWorker().getLinkedInputDirectories(), + configs.isAllowSymlinkTargetAbsolute(), removeDirectoryService, accessRecorder /* deadlineAfter=*/ diff --git a/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java b/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java index add913663..a23afaf0f 100644 --- a/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java +++ b/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java @@ -23,6 +23,7 @@ import static build.buildfarm.instance.server.AbstractServerInstance.INVALID_COMMAND; import static build.buildfarm.instance.server.AbstractServerInstance.OUTPUT_DIRECTORY_IS_OUTPUT_ANCESTOR; import static build.buildfarm.instance.server.AbstractServerInstance.OUTPUT_FILE_IS_OUTPUT_ANCESTOR; +import static build.buildfarm.instance.server.AbstractServerInstance.SYMLINK_TARGET_ABSOLUTE; import static com.google.common.truth.Truth.assertThat; import static com.google.common.util.concurrent.Futures.immediateFuture; import static org.mockito.Mockito.any; @@ -43,6 +44,7 @@ import build.bazel.remote.execution.v2.OutputDirectory; import build.bazel.remote.execution.v2.Platform; import build.bazel.remote.execution.v2.RequestMetadata; +import build.bazel.remote.execution.v2.SymlinkNode; import build.bazel.remote.execution.v2.Tree; import build.buildfarm.actioncache.ActionCache; import build.buildfarm.cas.ContentAddressableStorage; @@ -270,6 +272,7 @@ public void duplicateFileInputIsInvalid() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ Maps.newHashMap(), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFile=*/ file -> {}, /* onInputDirectorie=*/ directory -> {}, /* onInputDigest=*/ digest -> {}, @@ -304,6 +307,7 @@ public void duplicateEmptyDirectoryCheckPasses() throws StatusException { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ ImmutableMap.of(Digest.getDefaultInstance(), emptyDirectory), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, @@ -327,6 +331,7 @@ public void unsortedFileInputIsInvalid() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ Maps.newHashMap(), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, @@ -361,6 +366,7 @@ public void duplicateDirectoryInputIsInvalid() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ ImmutableMap.of(emptyDirectoryDigest, emptyDirectory), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, @@ -395,6 +401,7 @@ public void unsortedDirectoryInputIsInvalid() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ ImmutableMap.of(emptyDirectoryDigest, emptyDirectory), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, @@ -407,6 +414,48 @@ public void unsortedDirectoryInputIsInvalid() { assertThat(violation.getDescription()).isEqualTo(DIRECTORY_NOT_SORTED); } + @Test + public void shouldValidateIfSymlinkTargetAbsolute() { + // invalid for disallowed + PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); + Directory absoluteSymlinkDirectory = + Directory.newBuilder() + .addSymlinks(SymlinkNode.newBuilder().setName("foo").setTarget("/root/secret").build()) + .build(); + AbstractServerInstance.validateActionInputDirectory( + ACTION_INPUT_ROOT_DIRECTORY_PATH, + absoluteSymlinkDirectory, + /* pathDigests=*/ new Stack<>(), + /* visited=*/ Sets.newHashSet(), + /* directoriesIndex=*/ Maps.newHashMap(), + /* allowSymlinkTargetAbsolute=*/ false, + /* onInputFile=*/ file -> {}, + /* onInputDirectorie=*/ directory -> {}, + /* onInputDigest=*/ digest -> {}, + preconditionFailure); + + assertThat(preconditionFailure.getViolationsCount()).isEqualTo(1); + Violation violation = preconditionFailure.getViolationsList().get(0); + assertThat(violation.getType()).isEqualTo(VIOLATION_TYPE_INVALID); + assertThat(violation.getSubject()).isEqualTo("/: foo -> /root/secret"); + assertThat(violation.getDescription()).isEqualTo(SYMLINK_TARGET_ABSOLUTE); + + // valid for allowed + preconditionFailure = PreconditionFailure.newBuilder(); + AbstractServerInstance.validateActionInputDirectory( + ACTION_INPUT_ROOT_DIRECTORY_PATH, + absoluteSymlinkDirectory, + /* pathDigests=*/ new Stack<>(), + /* visited=*/ Sets.newHashSet(), + /* directoriesIndex=*/ Maps.newHashMap(), + /* allowSymlinkTargetAbsolute=*/ true, + /* onInputFile=*/ file -> {}, + /* onInputDirectorie=*/ directory -> {}, + /* onInputDigest=*/ digest -> {}, + preconditionFailure); + assertThat(preconditionFailure.getViolationsCount()).isEqualTo(0); + } + @Test public void nestedOutputDirectoriesAreInvalid() { PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); @@ -547,6 +596,7 @@ public void multipleIdenticalDirectoryMissingAreAllPreconditionFailures() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ ImmutableMap.of(), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, @@ -608,6 +658,7 @@ public void validationRevisitReplicatesPreconditionFailures() { /* pathDigests=*/ new Stack<>(), /* visited=*/ Sets.newHashSet(), /* directoriesIndex=*/ ImmutableMap.of(fooDigest, foo), + /* allowSymlinkTargetAbsolute=*/ false, /* onInputFiles=*/ file -> {}, /* onInputDirectories=*/ directory -> {}, /* onInputDigests=*/ digest -> {}, From df9ce1ddbd0a51b22eb535152794601125d59b84 Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Sun, 29 Oct 2023 14:05:45 -0700 Subject: [PATCH 13/68] chore: update bazel to 6.4.0 (#1513) Trying to get more info on the Lombok stamping issue on Windows CI. See also https://github.com/bazelbuild/bazel/issues/10363 and https://github.com/bazelbuild/bazel/pull/18185 --- .bazelversion | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.bazelversion b/.bazelversion index 5e3254243..19b860c18 100644 --- a/.bazelversion +++ b/.bazelversion @@ -1 +1 @@ -6.1.2 +6.4.0 From bd740c93510bc9c5abe2ba1860ae10bbbf4eae08 Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Mon, 30 Oct 2023 12:22:15 -0400 Subject: [PATCH 14/68] Rename instance types (#1514) --- .../src/main/resources/proto/buildfarm.proto | 8 +-- .../build/buildfarm/instance/server/BUILD | 2 +- ...tServerInstance.java => NodeInstance.java} | 11 ++-- .../shard/RemoteInputStreamFactory.java | 2 +- ...ShardInstance.java => ServerInstance.java} | 28 +++++----- .../buildfarm/server/BuildFarmServer.java | 6 +-- .../build/buildfarm/worker/shard/Worker.java | 5 +- ...orkerInstance.java => WorkerInstance.java} | 8 +-- ...nstanceTest.java => NodeInstanceTest.java} | 53 +++++++++---------- .../java/build/buildfarm/instance/shard/BUILD | 4 +- ...tanceTest.java => ServerInstanceTest.java} | 12 ++--- ...tanceTest.java => WorkerInstanceTest.java} | 6 +-- 12 files changed, 71 insertions(+), 74 deletions(-) rename src/main/java/build/buildfarm/instance/server/{AbstractServerInstance.java => NodeInstance.java} (99%) rename src/main/java/build/buildfarm/instance/shard/{ShardInstance.java => ServerInstance.java} (99%) rename src/main/java/build/buildfarm/worker/shard/{ShardWorkerInstance.java => WorkerInstance.java} (98%) rename src/test/java/build/buildfarm/instance/server/{AbstractServerInstanceTest.java => NodeInstanceTest.java} (94%) rename src/test/java/build/buildfarm/instance/shard/{ShardInstanceTest.java => ServerInstanceTest.java} (99%) rename src/test/java/build/buildfarm/worker/shard/{ShardWorkerInstanceTest.java => WorkerInstanceTest.java} (97%) diff --git a/admin/main/src/main/resources/proto/buildfarm.proto b/admin/main/src/main/resources/proto/buildfarm.proto index 8a000132a..d391847cb 100644 --- a/admin/main/src/main/resources/proto/buildfarm.proto +++ b/admin/main/src/main/resources/proto/buildfarm.proto @@ -520,7 +520,7 @@ message RedisShardBackplaneConfig { int32 max_attempts = 33; } -message ShardInstanceConfig { +message ServerInstanceConfig { bool run_dispatched_monitor = 1; int32 dispatched_monitor_interval_seconds = 2; @@ -544,7 +544,7 @@ message ShardInstanceConfig { google.protobuf.Duration grpc_timeout = 8; } -message ShardWorkerInstanceConfig { +message WorkerInstanceConfig { // whether to stream stdout from processes bool stream_stdout = 6; @@ -568,7 +568,7 @@ message ShardWorkerInstanceConfig { } message ShardWorkerConfig { - ShardWorkerInstanceConfig shard_worker_instance_config = 1; + WorkerInstanceConfig shard_worker_instance_config = 1; int32 port = 2; @@ -836,7 +836,7 @@ message InstanceConfig { oneof type { MemoryInstanceConfig memory_instance_config = 3; - ShardInstanceConfig shard_instance_config = 4; + ServerInstanceConfig shard_instance_config = 4; } } diff --git a/src/main/java/build/buildfarm/instance/server/BUILD b/src/main/java/build/buildfarm/instance/server/BUILD index ecff968e7..1c63e9896 100644 --- a/src/main/java/build/buildfarm/instance/server/BUILD +++ b/src/main/java/build/buildfarm/instance/server/BUILD @@ -1,8 +1,8 @@ java_library( name = "server", srcs = [ - "AbstractServerInstance.java", "GetDirectoryFunction.java", + "NodeInstance.java", "OperationsMap.java", "WatchFuture.java", ], diff --git a/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java b/src/main/java/build/buildfarm/instance/server/NodeInstance.java similarity index 99% rename from src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java rename to src/main/java/build/buildfarm/instance/server/NodeInstance.java index 541e13701..e44305f32 100644 --- a/src/main/java/build/buildfarm/instance/server/AbstractServerInstance.java +++ b/src/main/java/build/buildfarm/instance/server/NodeInstance.java @@ -144,7 +144,7 @@ import lombok.extern.java.Log; @Log -public abstract class AbstractServerInstance implements Instance { +public abstract class NodeInstance implements Instance { private final String name; protected final ContentAddressableStorage contentAddressableStorage; protected final ActionCache actionCache; @@ -229,7 +229,7 @@ public abstract class AbstractServerInstance implements Instance { public static final String NO_REQUEUE_COMPLETE_MESSAGE = "Operation %s not requeued. Operation has already completed."; - public AbstractServerInstance( + public NodeInstance( String name, DigestUtil digestUtil, ContentAddressableStorage contentAddressableStorage, @@ -1967,19 +1967,18 @@ public ServerCapabilities getCapabilities() { @Override public WorkerProfileMessage getWorkerProfile() { throw new UnsupportedOperationException( - "AbstractServerInstance doesn't support getWorkerProfile() method."); + "NodeInstance doesn't support getWorkerProfile() method."); } @Override public WorkerListMessage getWorkerList() { - throw new UnsupportedOperationException( - "AbstractServerInstance doesn't support getWorkerList() method."); + throw new UnsupportedOperationException("NodeInstance doesn't support getWorkerList() method."); } @Override public PrepareWorkerForGracefulShutDownRequestResults shutDownWorkerGracefully() { throw new UnsupportedOperationException( - "AbstractServerInstance doesn't support drainWorkerPipeline() method."); + "NodeInstance doesn't support drainWorkerPipeline() method."); } @Override diff --git a/src/main/java/build/buildfarm/instance/shard/RemoteInputStreamFactory.java b/src/main/java/build/buildfarm/instance/shard/RemoteInputStreamFactory.java index 640878707..8a00e9ae9 100644 --- a/src/main/java/build/buildfarm/instance/shard/RemoteInputStreamFactory.java +++ b/src/main/java/build/buildfarm/instance/shard/RemoteInputStreamFactory.java @@ -30,7 +30,7 @@ import build.buildfarm.common.DigestUtil; import build.buildfarm.common.InputStreamFactory; import build.buildfarm.instance.Instance; -import build.buildfarm.instance.shard.ShardInstance.WorkersCallback; +import build.buildfarm.instance.shard.ServerInstance.WorkersCallback; import com.google.common.base.Throwables; import com.google.common.cache.LoadingCache; import com.google.common.collect.Iterables; diff --git a/src/main/java/build/buildfarm/instance/shard/ShardInstance.java b/src/main/java/build/buildfarm/instance/shard/ServerInstance.java similarity index 99% rename from src/main/java/build/buildfarm/instance/shard/ShardInstance.java rename to src/main/java/build/buildfarm/instance/shard/ServerInstance.java index 241f1a8fa..ab865b0e0 100644 --- a/src/main/java/build/buildfarm/instance/shard/ShardInstance.java +++ b/src/main/java/build/buildfarm/instance/shard/ServerInstance.java @@ -81,7 +81,7 @@ import build.buildfarm.common.grpc.UniformDelegateServerCallStreamObserver; import build.buildfarm.instance.Instance; import build.buildfarm.instance.MatchListener; -import build.buildfarm.instance.server.AbstractServerInstance; +import build.buildfarm.instance.server.NodeInstance; import build.buildfarm.operations.EnrichedOperation; import build.buildfarm.operations.FindOperationsResults; import build.buildfarm.v1test.BackplaneStatus; @@ -167,7 +167,7 @@ import lombok.extern.java.Log; @Log -public class ShardInstance extends AbstractServerInstance { +public class ServerInstance extends NodeInstance { private static final ListenableFuture IMMEDIATE_VOID_FUTURE = Futures.immediateFuture(null); private static final String TIMEOUT_OUT_OF_BOUNDS = @@ -259,14 +259,14 @@ private static Backplane createBackplane(String identifier) throws Configuration identifier, /* subscribeToBackplane=*/ true, configs.getServer().isRunFailsafeOperation(), - ShardInstance::stripOperation, - ShardInstance::stripQueuedOperation); + ServerInstance::stripOperation, + ServerInstance::stripQueuedOperation); } else { throw new IllegalArgumentException("Shard Backplane not set in config"); } } - public ShardInstance(String name, String identifier, DigestUtil digestUtil, Runnable onStop) + public ServerInstance(String name, String identifier, DigestUtil digestUtil, Runnable onStop) throws InterruptedException, ConfigurationException { this( name, @@ -276,7 +276,7 @@ public ShardInstance(String name, String identifier, DigestUtil digestUtil, Runn /* actionCacheFetchService=*/ BuildfarmExecutors.getActionCacheFetchServicePool()); } - private ShardInstance( + private ServerInstance( String name, DigestUtil digestUtil, Backplane backplane, @@ -328,7 +328,7 @@ void initializeCaches() { .build(); } - public ShardInstance( + public ServerInstance( String name, DigestUtil digestUtil, Backplane backplane, @@ -1086,7 +1086,7 @@ public void onError(Throwable t) { backplane, workerSet, locationSet, - ShardInstance.this::workerStub, + ServerInstance.this::workerStub, blobDigest, directExecutor(), RequestMetadata.getDefaultInstance()), @@ -2251,7 +2251,7 @@ public ListenableFuture queue(ExecuteEntry executeEntry, Poller poller, Du log.log( Level.FINER, format( - "ShardInstance(%s): checkCache(%s): %sus elapsed", + "ServerInstance(%s): checkCache(%s): %sus elapsed", getName(), operation.getName(), checkCacheUSecs)); return IMMEDIATE_VOID_FUTURE; } @@ -2278,7 +2278,7 @@ private ListenableFuture transformAndQueue( log.log( Level.FINER, format( - "ShardInstance(%s): queue(%s): fetching action %s", + "ServerInstance(%s): queue(%s): fetching action %s", getName(), operation.getName(), actionDigest.getHash())); RequestMetadata requestMetadata = executeEntry.getRequestMetadata(); ListenableFuture actionFuture = @@ -2321,7 +2321,7 @@ private ListenableFuture transformAndQueue( log.log( Level.FINER, format( - "ShardInstance(%s): queue(%s): fetched action %s transforming queuedOperation", + "ServerInstance(%s): queue(%s): fetched action %s transforming queuedOperation", getName(), operation.getName(), actionDigest.getHash())); Stopwatch transformStopwatch = Stopwatch.createStarted(); return transform( @@ -2351,7 +2351,7 @@ private ListenableFuture transformAndQueue( log.log( Level.FINER, format( - "ShardInstance(%s): queue(%s): queuedOperation %s transformed, validating", + "ServerInstance(%s): queue(%s): queuedOperation %s transformed, validating", getName(), operation.getName(), DigestUtil.toString( @@ -2373,7 +2373,7 @@ private ListenableFuture transformAndQueue( log.log( Level.FINER, format( - "ShardInstance(%s): queue(%s): queuedOperation %s validated, uploading", + "ServerInstance(%s): queue(%s): queuedOperation %s validated, uploading", getName(), operation.getName(), DigestUtil.toString( @@ -2425,7 +2425,7 @@ public void onSuccess(ProfiledQueuedOperationMetadata profiledQueuedMetadata) { log.log( Level.FINER, format( - "ShardInstance(%s): queue(%s): %dus checkCache, %dus transform, %dus validate, %dus upload, %dus queue, %dus elapsed", + "ServerInstance(%s): queue(%s): %dus checkCache, %dus transform, %dus validate, %dus upload, %dus queue, %dus elapsed", getName(), queueOperation.getName(), checkCacheUSecs, diff --git a/src/main/java/build/buildfarm/server/BuildFarmServer.java b/src/main/java/build/buildfarm/server/BuildFarmServer.java index a20f4a6f7..8672c0df7 100644 --- a/src/main/java/build/buildfarm/server/BuildFarmServer.java +++ b/src/main/java/build/buildfarm/server/BuildFarmServer.java @@ -29,7 +29,7 @@ import build.buildfarm.common.services.ByteStreamService; import build.buildfarm.common.services.ContentAddressableStorageService; import build.buildfarm.instance.Instance; -import build.buildfarm.instance.shard.ShardInstance; +import build.buildfarm.instance.shard.ServerInstance; import build.buildfarm.metrics.prometheus.PrometheusPublisher; import build.buildfarm.server.services.ActionCacheService; import build.buildfarm.server.services.CapabilitiesService; @@ -109,9 +109,9 @@ public void prepareServerForGracefulShutdown() { } } - private ShardInstance createInstance() + private ServerInstance createInstance() throws IOException, ConfigurationException, InterruptedException { - return new ShardInstance( + return new ServerInstance( configs.getServer().getName(), configs.getServer().getSession() + "-" + configs.getServer().getName(), new DigestUtil(configs.getDigestFunction()), diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index eaf8b6cb1..fd556fc8e 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -123,7 +123,7 @@ public final class Worker extends LoggingMain { private boolean inGracefulShutdown = false; private boolean isPaused = false; - private ShardWorkerInstance instance; + private WorkerInstance instance; @SuppressWarnings("deprecation") private final HealthStatusManager healthStatusManager = new HealthStatusManager(); @@ -532,8 +532,7 @@ public void start() throws ConfigurationException, InterruptedException, IOExcep remoteInputStreamFactory, removeDirectoryService, accessRecorder, storage); instance = - new ShardWorkerInstance( - configs.getWorker().getPublicName(), digestUtil, backplane, storage); + new WorkerInstance(configs.getWorker().getPublicName(), digestUtil, backplane, storage); // Create the appropriate writer for the context CasWriter writer; diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerInstance.java b/src/main/java/build/buildfarm/worker/shard/WorkerInstance.java similarity index 98% rename from src/main/java/build/buildfarm/worker/shard/ShardWorkerInstance.java rename to src/main/java/build/buildfarm/worker/shard/WorkerInstance.java index a891af7fa..e100417f5 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerInstance.java +++ b/src/main/java/build/buildfarm/worker/shard/WorkerInstance.java @@ -36,7 +36,7 @@ import build.buildfarm.common.Write; import build.buildfarm.common.grpc.UniformDelegateServerCallStreamObserver; import build.buildfarm.instance.MatchListener; -import build.buildfarm.instance.server.AbstractServerInstance; +import build.buildfarm.instance.server.NodeInstance; import build.buildfarm.operations.EnrichedOperation; import build.buildfarm.operations.FindOperationsResults; import build.buildfarm.v1test.BackplaneStatus; @@ -68,13 +68,13 @@ import lombok.extern.java.Log; @Log -public class ShardWorkerInstance extends AbstractServerInstance { +public class WorkerInstance extends NodeInstance { private static final Counter IO_METRIC = Counter.build().name("io_bytes_read").help("Read I/O (bytes)").register(); private final Backplane backplane; - public ShardWorkerInstance( + public WorkerInstance( String name, DigestUtil digestUtil, Backplane backplane, @@ -346,7 +346,7 @@ protected static ExecuteOperationMetadata expectExecuteOperationMetadata(Operati return null; } } else { - return AbstractServerInstance.expectExecuteOperationMetadata(operation); + return NodeInstance.expectExecuteOperationMetadata(operation); } } diff --git a/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java b/src/test/java/build/buildfarm/instance/server/NodeInstanceTest.java similarity index 94% rename from src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java rename to src/test/java/build/buildfarm/instance/server/NodeInstanceTest.java index a23afaf0f..454440281 100644 --- a/src/test/java/build/buildfarm/instance/server/AbstractServerInstanceTest.java +++ b/src/test/java/build/buildfarm/instance/server/NodeInstanceTest.java @@ -17,13 +17,13 @@ import static build.buildfarm.common.Actions.checkPreconditionFailure; import static build.buildfarm.common.Errors.VIOLATION_TYPE_INVALID; import static build.buildfarm.common.Errors.VIOLATION_TYPE_MISSING; -import static build.buildfarm.instance.server.AbstractServerInstance.ACTION_INPUT_ROOT_DIRECTORY_PATH; -import static build.buildfarm.instance.server.AbstractServerInstance.DIRECTORY_NOT_SORTED; -import static build.buildfarm.instance.server.AbstractServerInstance.DUPLICATE_DIRENT; -import static build.buildfarm.instance.server.AbstractServerInstance.INVALID_COMMAND; -import static build.buildfarm.instance.server.AbstractServerInstance.OUTPUT_DIRECTORY_IS_OUTPUT_ANCESTOR; -import static build.buildfarm.instance.server.AbstractServerInstance.OUTPUT_FILE_IS_OUTPUT_ANCESTOR; -import static build.buildfarm.instance.server.AbstractServerInstance.SYMLINK_TARGET_ABSOLUTE; +import static build.buildfarm.instance.server.NodeInstance.ACTION_INPUT_ROOT_DIRECTORY_PATH; +import static build.buildfarm.instance.server.NodeInstance.DIRECTORY_NOT_SORTED; +import static build.buildfarm.instance.server.NodeInstance.DUPLICATE_DIRENT; +import static build.buildfarm.instance.server.NodeInstance.INVALID_COMMAND; +import static build.buildfarm.instance.server.NodeInstance.OUTPUT_DIRECTORY_IS_OUTPUT_ANCESTOR; +import static build.buildfarm.instance.server.NodeInstance.OUTPUT_FILE_IS_OUTPUT_ANCESTOR; +import static build.buildfarm.instance.server.NodeInstance.SYMLINK_TARGET_ABSOLUTE; import static com.google.common.truth.Truth.assertThat; import static com.google.common.util.concurrent.Futures.immediateFuture; import static org.mockito.Mockito.any; @@ -99,10 +99,10 @@ @RunWith(JUnit4.class) @Log -public class AbstractServerInstanceTest { +public class NodeInstanceTest { private static final DigestUtil DIGEST_UTIL = new DigestUtil(HashFunction.SHA256); - static class DummyServerInstance extends AbstractServerInstance { + static class DummyServerInstance extends NodeInstance { DummyServerInstance( ContentAddressableStorage contentAddressableStorage, ActionCache actionCache) { super( @@ -261,7 +261,7 @@ public PrepareWorkerForGracefulShutDownRequestResults shutDownWorkerGracefully() @Test public void duplicateFileInputIsInvalid() { PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, Directory.newBuilder() .addAllFiles( @@ -290,7 +290,7 @@ public void duplicateEmptyDirectoryCheckPasses() throws StatusException { Directory emptyDirectory = Directory.getDefaultInstance(); Digest emptyDirectoryDigest = DIGEST_UTIL.compute(emptyDirectory); PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, Directory.newBuilder() .addAllDirectories( @@ -320,7 +320,7 @@ public void duplicateEmptyDirectoryCheckPasses() throws StatusException { @Test public void unsortedFileInputIsInvalid() { PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, Directory.newBuilder() .addAllFiles( @@ -349,7 +349,7 @@ public void duplicateDirectoryInputIsInvalid() { Directory emptyDirectory = Directory.getDefaultInstance(); Digest emptyDirectoryDigest = DIGEST_UTIL.compute(emptyDirectory); PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, Directory.newBuilder() .addAllDirectories( @@ -384,7 +384,7 @@ public void unsortedDirectoryInputIsInvalid() { Directory emptyDirectory = Directory.getDefaultInstance(); Digest emptyDirectoryDigest = DIGEST_UTIL.compute(emptyDirectory); PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, Directory.newBuilder() .addAllDirectories( @@ -422,7 +422,7 @@ public void shouldValidateIfSymlinkTargetAbsolute() { Directory.newBuilder() .addSymlinks(SymlinkNode.newBuilder().setName("foo").setTarget("/root/secret").build()) .build(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, absoluteSymlinkDirectory, /* pathDigests=*/ new Stack<>(), @@ -442,7 +442,7 @@ public void shouldValidateIfSymlinkTargetAbsolute() { // valid for allowed preconditionFailure = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, absoluteSymlinkDirectory, /* pathDigests=*/ new Stack<>(), @@ -459,7 +459,7 @@ public void shouldValidateIfSymlinkTargetAbsolute() { @Test public void nestedOutputDirectoriesAreInvalid() { PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateOutputs( + NodeInstance.validateOutputs( ImmutableSet.of(), ImmutableSet.of(), ImmutableSet.of(), @@ -476,7 +476,7 @@ public void nestedOutputDirectoriesAreInvalid() { @Test public void outputDirectoriesContainingOutputFilesAreInvalid() { PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateOutputs( + NodeInstance.validateOutputs( ImmutableSet.of(), ImmutableSet.of(), ImmutableSet.of("foo/bar"), @@ -493,7 +493,7 @@ public void outputDirectoriesContainingOutputFilesAreInvalid() { @Test public void outputFilesAsOutputDirectoryAncestorsAreInvalid() { PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); - AbstractServerInstance.validateOutputs( + NodeInstance.validateOutputs( ImmutableSet.of(), ImmutableSet.of(), ImmutableSet.of("foo"), @@ -509,7 +509,7 @@ public void outputFilesAsOutputDirectoryAncestorsAreInvalid() { @Test public void emptyArgumentListIsInvalid() { - AbstractServerInstance instance = new DummyServerInstance(); + NodeInstance instance = new DummyServerInstance(); PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); instance.validateCommand( @@ -529,7 +529,7 @@ public void emptyArgumentListIsInvalid() { @Test public void absoluteWorkingDirectoryIsInvalid() { - AbstractServerInstance instance = new DummyServerInstance(); + NodeInstance instance = new DummyServerInstance(); PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); instance.validateCommand( @@ -549,7 +549,7 @@ public void absoluteWorkingDirectoryIsInvalid() { @Test public void undeclaredWorkingDirectoryIsInvalid() { - AbstractServerInstance instance = new DummyServerInstance(); + NodeInstance instance = new DummyServerInstance(); Digest inputRootDigest = DIGEST_UTIL.compute(Directory.getDefaultInstance()); PreconditionFailure.Builder preconditionFailureBuilder = PreconditionFailure.newBuilder(); @@ -590,7 +590,7 @@ public void multipleIdenticalDirectoryMissingAreAllPreconditionFailures() { .setDigest(missingDirectoryDigest) .build())) .build(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, root, /* pathDigests=*/ new Stack<>(), @@ -652,7 +652,7 @@ public void validationRevisitReplicatesPreconditionFailures() { DirectoryNode.newBuilder().setName("bar").setDigest(fooDigest).build(), DirectoryNode.newBuilder().setName("foo").setDigest(fooDigest).build())) .build(); - AbstractServerInstance.validateActionInputDirectory( + NodeInstance.validateActionInputDirectory( ACTION_INPUT_ROOT_DIRECTORY_PATH, root, /* pathDigests=*/ new Stack<>(), @@ -721,8 +721,7 @@ public void outputDirectoriesFilesAreEnsuredPresent() throws Exception { .build(); ContentAddressableStorage contentAddressableStorage = mock(ContentAddressableStorage.class); ActionCache actionCache = mock(ActionCache.class); - AbstractServerInstance instance = - new DummyServerInstance(contentAddressableStorage, actionCache); + NodeInstance instance = new DummyServerInstance(contentAddressableStorage, actionCache); Tree tree = Tree.newBuilder() @@ -782,7 +781,7 @@ public void fetchBlobWriteCompleteIsSuccess() throws Exception { Digest expectedDigest = contentDigest.toBuilder().setSizeBytes(-1).build(); ContentAddressableStorage contentAddressableStorage = mock(ContentAddressableStorage.class); - AbstractServerInstance instance = new DummyServerInstance(contentAddressableStorage, null); + NodeInstance instance = new DummyServerInstance(contentAddressableStorage, null); RequestMetadata requestMetadata = RequestMetadata.getDefaultInstance(); Write write = mock(Write.class); diff --git a/src/test/java/build/buildfarm/instance/shard/BUILD b/src/test/java/build/buildfarm/instance/shard/BUILD index cd0ae18d2..b635c74ba 100644 --- a/src/test/java/build/buildfarm/instance/shard/BUILD +++ b/src/test/java/build/buildfarm/instance/shard/BUILD @@ -110,10 +110,10 @@ java_test( ) java_test( - name = "ShardInstanceTest", + name = "ServerInstanceTest", size = "small", srcs = [ - "ShardInstanceTest.java", + "ServerInstanceTest.java", "UnobservableWatcher.java", ], data = ["//examples:example_configs"], diff --git a/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java b/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java similarity index 99% rename from src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java rename to src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java index aa1adc5d3..cd959950a 100644 --- a/src/test/java/build/buildfarm/instance/shard/ShardInstanceTest.java +++ b/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java @@ -20,9 +20,9 @@ import static build.buildfarm.common.Actions.invalidActionVerboseMessage; import static build.buildfarm.common.Errors.VIOLATION_TYPE_INVALID; import static build.buildfarm.common.Errors.VIOLATION_TYPE_MISSING; -import static build.buildfarm.instance.server.AbstractServerInstance.INVALID_PLATFORM; -import static build.buildfarm.instance.server.AbstractServerInstance.MISSING_ACTION; -import static build.buildfarm.instance.server.AbstractServerInstance.MISSING_COMMAND; +import static build.buildfarm.instance.server.NodeInstance.INVALID_PLATFORM; +import static build.buildfarm.instance.server.NodeInstance.MISSING_ACTION; +import static build.buildfarm.instance.server.NodeInstance.MISSING_COMMAND; import static com.google.common.base.Predicates.notNull; import static com.google.common.truth.Truth.assertThat; import static com.google.common.util.concurrent.Futures.immediateFuture; @@ -121,14 +121,14 @@ import org.mockito.stubbing.Answer; @RunWith(JUnit4.class) -public class ShardInstanceTest { +public class ServerInstanceTest { private static final DigestUtil DIGEST_UTIL = new DigestUtil(HashFunction.SHA256); private static final long QUEUE_TEST_TIMEOUT_SECONDS = 3; private static final Duration DEFAULT_TIMEOUT = Durations.fromSeconds(60); private static final Command SIMPLE_COMMAND = Command.newBuilder().addAllArguments(ImmutableList.of("true")).build(); - private ShardInstance instance; + private ServerInstance instance; private Map blobDigests; @Mock private Backplane mockBackplane; @@ -145,7 +145,7 @@ public void setUp() throws InterruptedException { blobDigests = Maps.newHashMap(); ActionCache actionCache = new ShardActionCache(10, mockBackplane, newDirectExecutorService()); instance = - new ShardInstance( + new ServerInstance( "shard", DIGEST_UTIL, mockBackplane, diff --git a/src/test/java/build/buildfarm/worker/shard/ShardWorkerInstanceTest.java b/src/test/java/build/buildfarm/worker/shard/WorkerInstanceTest.java similarity index 97% rename from src/test/java/build/buildfarm/worker/shard/ShardWorkerInstanceTest.java rename to src/test/java/build/buildfarm/worker/shard/WorkerInstanceTest.java index 7a4e40be2..3df73187b 100644 --- a/src/test/java/build/buildfarm/worker/shard/ShardWorkerInstanceTest.java +++ b/src/test/java/build/buildfarm/worker/shard/WorkerInstanceTest.java @@ -50,19 +50,19 @@ import org.mockito.MockitoAnnotations; @RunWith(JUnit4.class) -public class ShardWorkerInstanceTest { +public class WorkerInstanceTest { private final DigestUtil DIGEST_UTIL = new DigestUtil(HashFunction.SHA256); @Mock private Backplane backplane; @Mock private ContentAddressableStorage storage; - private ShardWorkerInstance instance; + private WorkerInstance instance; @Before public void setUp() throws Exception { MockitoAnnotations.initMocks(this); - instance = new ShardWorkerInstance("test", DIGEST_UTIL, backplane, storage); + instance = new WorkerInstance("test", DIGEST_UTIL, backplane, storage); } @SuppressWarnings("unchecked") From 2a61f77dc1713f276674bdbc9a839ec2732feb07 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Mon, 30 Oct 2023 14:48:00 -0400 Subject: [PATCH 15/68] Create SymlinkNode outputs during upload (#1515) Default disabled, available with createSymlinkOutputs option in Worker config. --- _site/docs/configuration/configuration.md | 1 + examples/config.yml | 1 + .../build/buildfarm/common/config/Worker.java | 1 + .../worker/shard/ShardWorkerContext.java | 38 +++++++++++++++++-- 4 files changed, 38 insertions(+), 3 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index ccfc8e026..466375373 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -262,6 +262,7 @@ backplane: | errorOperationRemainingResources | boolean, _false_ | | | | realInputDirectories | List of Strings, _external_ | | A list of paths that will not be subject to the effects of linkInputDirectories setting, may also be used to provide writable directories as input roots for actions which expect to be able to write to an input location and will fail if they cannot | | gracefulShutdownSeconds | Integer, 0 | | Time in seconds to allow for operations in flight to finish when shutdown signal is received | +| createSymlinkOutputs | boolean, _false_ | | Creates SymlinkNodes for symbolic links discovered in output paths for actions. No verification of the symlink target path occurs. Buildstream, for example, requires this. | ```yaml worker: diff --git a/examples/config.yml b/examples/config.yml index 5229fcd07..008bf2e1c 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -131,6 +131,7 @@ worker: alwaysUse: false selectForBlockNetwork: false selectForTmpFs: false + createSymlinkOutputs: false executionPolicies: - name: test executionWrapper: diff --git a/src/main/java/build/buildfarm/common/config/Worker.java b/src/main/java/build/buildfarm/common/config/Worker.java index e987c1e37..c446134fe 100644 --- a/src/main/java/build/buildfarm/common/config/Worker.java +++ b/src/main/java/build/buildfarm/common/config/Worker.java @@ -39,6 +39,7 @@ public class Worker { private int gracefulShutdownSeconds = 0; private ExecutionPolicy[] executionPolicies = {}; private SandboxSettings sandboxSettings = new SandboxSettings(); + private boolean createSymlinkOutputs = false; // These limited resources are only for the individual worker. // An example would be hardware resources such as GPUs. diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index c844f5396..98474b439 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -31,6 +31,7 @@ import build.bazel.remote.execution.v2.ExecutionStage; import build.bazel.remote.execution.v2.FileNode; import build.bazel.remote.execution.v2.Platform; +import build.bazel.remote.execution.v2.SymlinkNode; import build.bazel.remote.execution.v2.Tree; import build.buildfarm.backplane.Backplane; import build.buildfarm.common.CommandUtils; @@ -572,6 +573,7 @@ private void uploadOutputFile( static class OutputDirectoryContext { private final List files = new ArrayList<>(); private final List directories = new ArrayList<>(); + private final List symlinks = new ArrayList<>(); void addFile(FileNode fileNode) { files.add(fileNode); @@ -581,10 +583,19 @@ void addDirectory(DirectoryNode directoryNode) { directories.add(directoryNode); } + void addSymlink(SymlinkNode symlinkNode) { + symlinks.add(symlinkNode); + } + Directory toDirectory() { files.sort(Comparator.comparing(FileNode::getName)); directories.sort(Comparator.comparing(DirectoryNode::getName)); - return Directory.newBuilder().addAllFiles(files).addAllDirectories(directories).build(); + symlinks.sort(Comparator.comparing(SymlinkNode::getName)); + return Directory.newBuilder() + .addAllFiles(files) + .addAllDirectories(directories) + .addAllSymlinks(symlinks) + .build(); } } @@ -621,8 +632,30 @@ private void uploadOutputDirectory( @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + if (configs.getWorker().isCreateSymlinkOutputs() && attrs.isSymbolicLink()) { + visitSymbolicLink(file); + } else { + visitRegularFile(file, attrs); + } + return FileVisitResult.CONTINUE; + } + + private void visitSymbolicLink(Path file) throws IOException { + // TODO convert symlinks with absolute targets within execution root to relative ones + currentDirectory.addSymlink( + SymlinkNode.newBuilder() + .setName(file.getFileName().toString()) + .setTarget(Files.readSymbolicLink(file).toString()) + .build()); + } + + private void visitRegularFile(Path file, BasicFileAttributes attrs) throws IOException { Digest digest; try { + // should we create symlink nodes in output? + // is buildstream trying to execute in a specific container?? + // can get to NSFE for nonexistent symlinks + // can fail outright for a symlink to a directory digest = getDigestUtil().compute(file); } catch (NoSuchFileException e) { log.log( @@ -631,7 +664,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) "error visiting file %s under output dir %s", outputDirPath.relativize(file), outputDirPath.toAbsolutePath()), e); - return FileVisitResult.CONTINUE; + return; } // should we cast to PosixFilePermissions and do gymnastics there for executable? @@ -655,7 +688,6 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) .setDescription( "An output could not be uploaded because it exceeded the maximum size of an entry"); } - return FileVisitResult.CONTINUE; } @Override From 76c2657111e5ebe92bdc99195a892195aa992be0 Mon Sep 17 00:00:00 2001 From: amishra-u <119983081+amishra-u@users.noreply.github.com> Date: Mon, 30 Oct 2023 19:26:18 -0700 Subject: [PATCH 16/68] feat: Implement CAS lease extension (#1455) Problem Enabling the findMissingBlobsViaBackplane flag in BuildfarmServer eliminates the need for the BuildfarmWorker's fmb API call. This BuildfarmWorker:fmb call was also responsible for tracking CAS entry access. As result, our CAS cache eviction strategy shifted from LRU to FIFO. When the findMissingBlobsViaBackplane flag is enabled, the buildfarm relies on the backplane as the definitive source for CAS availability. Since we don't update CAS expiry on each access, the backplane will independently expire CAS entries based on the specified cas_expire duration, even if they are actively being read. Solution Updated bfServer:fmb call to perform non-blocking fmb calls to workers, allowing these workers to record access for the relevant CAS entries. Extended expiry duration for available CAS entries in the backplane on each fmb call. With these changes, we can utilize Bazel's experimental_remote_cache_lease_extension and experimental_remote_cache_ttl flags for incremental builds. Closes #1428 --- defs.bzl | 1 + .../build/buildfarm/backplane/Backplane.java | 3 + .../instance/shard/CasWorkerMap.java | 7 +++ .../instance/shard/JedisCasWorkerMap.java | 11 ++++ .../instance/shard/RedisShardBackplane.java | 5 ++ .../instance/shard/RedissonCasWorkerMap.java | 8 +++ .../instance/shard/ServerInstance.java | 57 ++++++++++++++--- .../java/build/buildfarm/instance/shard/BUILD | 19 ++++++ .../instance/shard/JedisCasWorkerMapTest.java | 63 +++++++++++++++++++ .../instance/shard/ServerInstanceTest.java | 14 +++++ 10 files changed, 180 insertions(+), 8 deletions(-) create mode 100644 src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java diff --git a/defs.bzl b/defs.bzl index 5b0e4f640..4b0a5e49b 100644 --- a/defs.bzl +++ b/defs.bzl @@ -60,6 +60,7 @@ def buildfarm_init(name = "buildfarm"): "com.fasterxml.jackson.core:jackson-databind:2.15.0", "com.github.ben-manes.caffeine:caffeine:2.9.0", "com.github.docker-java:docker-java:3.2.11", + "com.github.fppt:jedis-mock:1.0.10", "com.github.jnr:jffi:1.2.16", "com.github.jnr:jffi:jar:native:1.2.16", "com.github.jnr:jnr-constants:0.9.9", diff --git a/src/main/java/build/buildfarm/backplane/Backplane.java b/src/main/java/build/buildfarm/backplane/Backplane.java index 557a7dc16..e13616b32 100644 --- a/src/main/java/build/buildfarm/backplane/Backplane.java +++ b/src/main/java/build/buildfarm/backplane/Backplane.java @@ -278,4 +278,7 @@ boolean pollOperation(QueueEntry queueEntry, ExecutionStage.Value stage, long re Boolean propertiesEligibleForQueue(List provisions); GetClientStartTimeResult getClientStartTime(GetClientStartTimeRequest request) throws IOException; + + /** Set expiry time for digests */ + void updateDigestsExpiry(Iterable digests) throws IOException; } diff --git a/src/main/java/build/buildfarm/instance/shard/CasWorkerMap.java b/src/main/java/build/buildfarm/instance/shard/CasWorkerMap.java index 794b296a1..55b2933e4 100644 --- a/src/main/java/build/buildfarm/instance/shard/CasWorkerMap.java +++ b/src/main/java/build/buildfarm/instance/shard/CasWorkerMap.java @@ -121,4 +121,11 @@ Map> getMap(RedisClient client, Iterable blobDigests * @note Suggested return identifier: mapSize. */ int size(RedisClient client) throws IOException; + + /** + * @brief Set the expiry duration for the digests. + * @param client Client used for interacting with redis when not using cacheMap. + * @param blobDigests The blob digests to set new the expiry duration. + */ + void setExpire(RedisClient client, Iterable blobDigests) throws IOException; } diff --git a/src/main/java/build/buildfarm/instance/shard/JedisCasWorkerMap.java b/src/main/java/build/buildfarm/instance/shard/JedisCasWorkerMap.java index d035d1049..65d1f06b8 100644 --- a/src/main/java/build/buildfarm/instance/shard/JedisCasWorkerMap.java +++ b/src/main/java/build/buildfarm/instance/shard/JedisCasWorkerMap.java @@ -234,6 +234,17 @@ public int size(RedisClient client) throws IOException { return client.call(jedis -> ScanCount.get(jedis, name + ":*", 1000)); } + @Override + public void setExpire(RedisClient client, Iterable blobDigests) throws IOException { + client.run( + jedis -> { + for (Digest blobDigest : blobDigests) { + String key = redisCasKey(blobDigest); + jedis.expire(key, keyExpiration_s); + } + }); + } + /** * @brief Get the redis key name. * @details This is to be used for the direct redis implementation. diff --git a/src/main/java/build/buildfarm/instance/shard/RedisShardBackplane.java b/src/main/java/build/buildfarm/instance/shard/RedisShardBackplane.java index ebb832f5a..535473f2a 100644 --- a/src/main/java/build/buildfarm/instance/shard/RedisShardBackplane.java +++ b/src/main/java/build/buildfarm/instance/shard/RedisShardBackplane.java @@ -1501,4 +1501,9 @@ public GetClientStartTimeResult getClientStartTime(GetClientStartTimeRequest req } return GetClientStartTimeResult.newBuilder().addAllClientStartTime(startTimes).build(); } + + @Override + public void updateDigestsExpiry(Iterable digests) throws IOException { + state.casWorkerMap.setExpire(client, digests); + } } diff --git a/src/main/java/build/buildfarm/instance/shard/RedissonCasWorkerMap.java b/src/main/java/build/buildfarm/instance/shard/RedissonCasWorkerMap.java index 234e15fb1..52b010c31 100644 --- a/src/main/java/build/buildfarm/instance/shard/RedissonCasWorkerMap.java +++ b/src/main/java/build/buildfarm/instance/shard/RedissonCasWorkerMap.java @@ -209,6 +209,14 @@ public int size(RedisClient client) { return cacheMap.size(); } + @Override + public void setExpire(RedisClient client, Iterable blobDigests) { + for (Digest blobDigest : blobDigests) { + String key = cacheMapCasKey(blobDigest); + cacheMap.expireKey(key, keyExpiration_s, TimeUnit.SECONDS); + } + } + /** * @brief Get a random element from the set. * @details Assumes the set is not empty. diff --git a/src/main/java/build/buildfarm/instance/shard/ServerInstance.java b/src/main/java/build/buildfarm/instance/shard/ServerInstance.java index ab865b0e0..b1cbed0e0 100644 --- a/src/main/java/build/buildfarm/instance/shard/ServerInstance.java +++ b/src/main/java/build/buildfarm/instance/shard/ServerInstance.java @@ -135,10 +135,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.time.Instant; +import java.util.AbstractMap; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; import java.util.Deque; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -660,7 +662,7 @@ public ListenableFuture> findMissingBlobs( } if (configs.getServer().isFindMissingBlobsViaBackplane()) { - return findMissingBlobsViaBackplane(nonEmptyDigests); + return findMissingBlobsViaBackplane(nonEmptyDigests, requestMetadata); } return findMissingBlobsQueryingEachWorker(nonEmptyDigests, requestMetadata); @@ -727,24 +729,40 @@ private ListenableFuture> findMissingBlobsQueryingEachWorker( // out-of-date and the server lies about which blobs are actually present. We provide this // alternative strategy for calculating missing blobs. private ListenableFuture> findMissingBlobsViaBackplane( - Iterable nonEmptyDigests) { + Iterable nonEmptyDigests, RequestMetadata requestMetadata) { try { Set uniqueDigests = new HashSet<>(); nonEmptyDigests.forEach(uniqueDigests::add); Map> foundBlobs = backplane.getBlobDigestsWorkers(uniqueDigests); Set workerSet = backplane.getStorageWorkers(); Map workersStartTime = backplane.getWorkersStartTimeInEpochSecs(workerSet); - return immediateFuture( + Map> digestAndWorkersMap = uniqueDigests.stream() - .filter( // best effort to present digests only missing on active workers + .map( digest -> { Set initialWorkers = foundBlobs.getOrDefault(digest, Collections.emptySet()); - return filterAndAdjustWorkersForDigest( - digest, initialWorkers, workerSet, workersStartTime) - .isEmpty(); + return new AbstractMap.SimpleEntry<>( + digest, + filterAndAdjustWorkersForDigest( + digest, initialWorkers, workerSet, workersStartTime)); }) - .collect(Collectors.toList())); + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + ListenableFuture> missingDigestFuture = + immediateFuture( + digestAndWorkersMap.entrySet().stream() + .filter(entry -> entry.getValue().isEmpty()) + .map(Map.Entry::getKey) + .collect(Collectors.toList())); + return transformAsync( + missingDigestFuture, + (missingDigest) -> { + extendLeaseForDigests(digestAndWorkersMap, requestMetadata); + return immediateFuture(missingDigest); + }, + // Propagate context values but don't cascade its cancellation for downstream calls. + Context.current().fork().fixedContextExecutor(directExecutor())); } catch (Exception e) { log.log(Level.SEVERE, "find missing blob via backplane failed", e); return immediateFailedFuture(Status.fromThrowable(e).asException()); @@ -789,6 +807,29 @@ private Set filterAndAdjustWorkersForDigest( return workersStartedBeforeDigestInsertion; } + private void extendLeaseForDigests( + Map> digestAndWorkersMap, RequestMetadata requestMetadata) { + Map> workerAndDigestMap = new HashMap<>(); + digestAndWorkersMap.forEach( + (digest, workers) -> + workers.forEach( + worker -> + workerAndDigestMap.computeIfAbsent(worker, w -> new HashSet<>()).add(digest))); + + workerAndDigestMap.forEach( + (worker, digests) -> workerStub(worker).findMissingBlobs(digests, requestMetadata)); + + try { + backplane.updateDigestsExpiry(digestAndWorkersMap.keySet()); + } catch (IOException e) { + log.log( + Level.WARNING, + format( + "Failed to update expiry duration for digests (%s) insertion time", + digestAndWorkersMap.keySet())); + } + } + private void findMissingBlobsOnWorker( String requestId, Iterable blobDigests, diff --git a/src/test/java/build/buildfarm/instance/shard/BUILD b/src/test/java/build/buildfarm/instance/shard/BUILD index b635c74ba..2f08b1003 100644 --- a/src/test/java/build/buildfarm/instance/shard/BUILD +++ b/src/test/java/build/buildfarm/instance/shard/BUILD @@ -219,3 +219,22 @@ java_test( "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", ], ) + +java_test( + name = "JedisCasWorkerMapTest", + size = "small", + srcs = [ + "JedisCasWorkerMapTest.java", + ], + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/redis", + "//src/main/java/build/buildfarm/instance/shard", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "//third_party/jedis", + "@maven//:com_github_fppt_jedis_mock", + "@maven//:com_google_truth_truth", + ], +) diff --git a/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java b/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java new file mode 100644 index 000000000..caa69536c --- /dev/null +++ b/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java @@ -0,0 +1,63 @@ +package build.buildfarm.instance.shard; + +import static com.google.common.truth.Truth.assertThat; + +import build.bazel.remote.execution.v2.Digest; +import build.buildfarm.common.DigestUtil; +import build.buildfarm.common.redis.RedisClient; +import com.github.fppt.jedismock.RedisServer; +import com.github.fppt.jedismock.server.ServiceOptions; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.JedisCluster; + +@RunWith(JUnit4.class) +public class JedisCasWorkerMapTest { + + private static final String CAS_PREFIX = "ContentAddressableStorage"; + + private RedisServer redisServer; + private RedisClient redisClient; + private JedisCasWorkerMap jedisCasWorkerMap; + + @Before + public void setup() throws IOException { + redisServer = + RedisServer.newRedisServer() + .setOptions(ServiceOptions.defaultOptions().withClusterModeEnabled()) + .start(); + redisClient = + new RedisClient( + new JedisCluster( + Collections.singleton( + new HostAndPort(redisServer.getHost(), redisServer.getBindPort())))); + jedisCasWorkerMap = new JedisCasWorkerMap(CAS_PREFIX, 60); + } + + @Test + public void testSetExpire() throws IOException { + Digest testDigest1 = Digest.newBuilder().setHash("abc").build(); + Digest testDigest2 = Digest.newBuilder().setHash("xyz").build(); + + String casKey1 = CAS_PREFIX + ":" + DigestUtil.toString(testDigest1); + String casKey2 = CAS_PREFIX + ":" + DigestUtil.toString(testDigest2); + + redisClient.run(jedis -> jedis.sadd(casKey1, "worker1")); + jedisCasWorkerMap.setExpire(redisClient, Arrays.asList(testDigest1, testDigest2)); + + assertThat((Long) redisClient.call(jedis -> jedis.ttl(casKey1))).isGreaterThan(0L); + assertThat((Long) redisClient.call(jedis -> jedis.ttl(casKey2))).isEqualTo(-2L); + } + + @After + public void tearDown() throws IOException { + redisServer.stop(); + } +} diff --git a/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java b/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java index cd959950a..d66a600da 100644 --- a/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java +++ b/src/test/java/build/buildfarm/instance/shard/ServerInstanceTest.java @@ -31,9 +31,13 @@ import static java.util.concurrent.Executors.newSingleThreadExecutor; import static java.util.concurrent.TimeUnit.SECONDS; import static org.mockito.AdditionalAnswers.answer; +import static org.mockito.ArgumentMatchers.anyIterable; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; @@ -80,6 +84,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.longrunning.Operation; import com.google.protobuf.Any; @@ -95,6 +100,7 @@ import io.grpc.stub.ServerCallStreamObserver; import io.grpc.stub.StreamObserver; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; @@ -1110,8 +1116,12 @@ public void findMissingBlobsTest_ViaBackPlane() throws Exception { buildfarmConfigs.getServer().setFindMissingBlobsViaBackplane(true); Set activeAndImposterWorkers = Sets.newHashSet(Iterables.concat(activeWorkers, imposterWorkers)); + when(mockBackplane.getStorageWorkers()).thenReturn(activeAndImposterWorkers); when(mockBackplane.getBlobDigestsWorkers(any(Iterable.class))).thenReturn(digestAndWorkersMap); + when(mockInstanceLoader.load(anyString())).thenReturn(mockWorkerInstance); + when(mockWorkerInstance.findMissingBlobs(anyIterable(), any(RequestMetadata.class))) + .thenReturn(Futures.immediateFuture(new ArrayList<>())); long serverStartTime = 1686951033L; // june 15th, 2023 Map workersStartTime = new HashMap<>(); @@ -1134,6 +1144,10 @@ public void findMissingBlobsTest_ViaBackPlane() throws Exception { Iterables.concat(missingDigests, digestAvailableOnImposters); assertThat(actualMissingDigests).containsExactlyElementsIn(expectedMissingDigests); + verify(mockWorkerInstance, atMost(3)) + .findMissingBlobs(anyIterable(), any(RequestMetadata.class)); + verify(mockWorkerInstance, atLeast(1)) + .findMissingBlobs(anyIterable(), any(RequestMetadata.class)); for (Digest digest : actualMissingDigests) { assertThat(digest).isNotIn(availableDigests); From cfa2e18724914034eb19c4c01f293117610fa0b7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 31 Oct 2023 14:48:35 -0400 Subject: [PATCH 17/68] Bump org.json:json from 20230227 to 20231013 in /admin/main (#1516) Bumps [org.json:json](https://github.com/douglascrockford/JSON-java) from 20230227 to 20231013. - [Release notes](https://github.com/douglascrockford/JSON-java/releases) - [Changelog](https://github.com/stleary/JSON-java/blob/master/docs/RELEASES.md) - [Commits](https://github.com/douglascrockford/JSON-java/commits) --- updated-dependencies: - dependency-name: org.json:json dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- admin/main/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/admin/main/pom.xml b/admin/main/pom.xml index 006217ccb..8e128ee64 100644 --- a/admin/main/pom.xml +++ b/admin/main/pom.xml @@ -94,7 +94,7 @@ org.json json - 20230227 + 20231013 org.projectlombok From ff00c8f3977aadf3df3febaceb07e575821ea9e2 Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Wed, 1 Nov 2023 16:14:45 -0400 Subject: [PATCH 18/68] Re-add missing graceful shutdown functionality (#1520) --- .../buildfarm/server/BuildFarmServer.java | 8 ++-- .../build/buildfarm/worker/shard/Worker.java | 47 +++++++++++++++++++ 2 files changed, 51 insertions(+), 4 deletions(-) diff --git a/src/main/java/build/buildfarm/server/BuildFarmServer.java b/src/main/java/build/buildfarm/server/BuildFarmServer.java index 8672c0df7..c44896320 100644 --- a/src/main/java/build/buildfarm/server/BuildFarmServer.java +++ b/src/main/java/build/buildfarm/server/BuildFarmServer.java @@ -88,20 +88,20 @@ public class BuildFarmServer extends LoggingMain { */ public void prepareServerForGracefulShutdown() { if (configs.getServer().getGracefulShutdownSeconds() == 0) { - System.err.println( + log.info( String.format("Graceful Shutdown is not enabled. Server is shutting down immediately.")); } else { try { - System.err.println( + log.info( String.format( "Graceful Shutdown - Waiting %d to allow connections to drain.", configs.getServer().getGracefulShutdownSeconds())); SECONDS.sleep(configs.getServer().getGracefulShutdownSeconds()); } catch (InterruptedException e) { - System.err.println( + log.info( "Graceful Shutdown - The server graceful shutdown is interrupted: " + e.getMessage()); } finally { - System.err.println( + log.info( String.format( "Graceful Shutdown - It took the server %d seconds to shutdown", configs.getServer().getGracefulShutdownSeconds())); diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index fd556fc8e..ce69eb3d1 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -137,6 +137,52 @@ public final class Worker extends LoggingMain { private LoadingCache workerStubs; private AtomicBoolean released = new AtomicBoolean(true); + /** + * The method will prepare the worker for graceful shutdown when the worker is ready. Note on + * using stderr here instead of log. By the time this is called in PreDestroy, the log is no + * longer available and is not logging messages. + */ + public void prepareWorkerForGracefulShutdown() { + if (configs.getWorker().getGracefulShutdownSeconds() == 0) { + log.info( + String.format( + "Graceful Shutdown is not enabled. Worker is shutting down without finishing executions in progress.")); + } else { + inGracefulShutdown = true; + log.info( + "Graceful Shutdown - The current worker will not be registered again and should be shutdown gracefully!"); + pipeline.stopMatchingOperations(); + int scanRate = 30; // check every 30 seconds + int timeWaited = 0; + int timeOut = configs.getWorker().getGracefulShutdownSeconds(); + try { + if (pipeline.isEmpty()) { + log.info("Graceful Shutdown - no work in the pipeline."); + } else { + log.info(String.format("Graceful Shutdown - waiting for executions to finish.")); + } + while (!pipeline.isEmpty() && timeWaited < timeOut) { + SECONDS.sleep(scanRate); + timeWaited += scanRate; + log.info( + String.format( + "Graceful Shutdown - Pipeline is still not empty after %d seconds.", timeWaited)); + } + } catch (InterruptedException e) { + log.info( + "Graceful Shutdown - The worker gracefully shutdown is interrupted: " + e.getMessage()); + } finally { + log.info( + String.format( + "Graceful Shutdown - It took the worker %d seconds to %s", + timeWaited, + pipeline.isEmpty() + ? "finish all actions" + : "gracefully shutdown but still cannot finish all actions")); + } + } + } + private Worker() { super("BuildFarmShardWorker"); } @@ -626,6 +672,7 @@ public synchronized void stop() throws InterruptedException { private void shutdown() throws InterruptedException { log.info("*** shutting down gRPC server since JVM is shutting down"); + prepareWorkerForGracefulShutdown(); PrometheusPublisher.stopHttpServer(); boolean interrupted = Thread.interrupted(); if (pipeline != null) { From afb06039ae368fa610c6d200b050f14ab0cffd02 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Fri, 14 Jul 2023 08:40:34 -0400 Subject: [PATCH 19/68] Technically correct to unwrap EE on lock failure --- src/main/java/build/buildfarm/cas/cfc/CASFileCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java index 21c90e5fe..56e53c3b2 100644 --- a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java +++ b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java @@ -1917,7 +1917,7 @@ private Entry safeStorageInsertion(String key, Entry entry) { lock = keyLocks.get(key); } catch (ExecutionException e) { // impossible without exception instantiating lock - throw new RuntimeException(e); + throw new RuntimeException(e.getCause()); } lock.lock(); From 9b5ec4340134b4a2825a712744b2d8855f7f80b0 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Tue, 17 Oct 2023 00:40:19 -0400 Subject: [PATCH 20/68] Bump rules_oss_audit and patch for py3.11 --- deps.bzl | 8 +++++--- third_party/rules_oss_audit_pyyaml.patch | 7 +++++++ 2 files changed, 12 insertions(+), 3 deletions(-) create mode 100644 third_party/rules_oss_audit_pyyaml.patch diff --git a/deps.bzl b/deps.bzl index e2d2609ac..55b12ab13 100644 --- a/deps.bzl +++ b/deps.bzl @@ -151,9 +151,11 @@ def archive_dependencies(third_party): }, { "name": "rules_oss_audit", - "sha256": "02962810bcf82d0c66f929ccc163423f53773b8b154574ca956345523243e70d", - "strip_prefix": "rules_oss_audit-1b2690cefd5a960c181e0d89bf3c076294a0e6f4", - "url": "https://github.com/vmware/rules_oss_audit/archive/1b2690cefd5a960c181e0d89bf3c076294a0e6f4.zip", + "sha256": "8ee8376b05b5ddd2287b070e9a88ec85ef907d47f44e321ce5d4bc2b192eed4e", + "strip_prefix": "rules_oss_audit-167dab5b16abdb5996438f22364de544ff24693f", + "url": "https://github.com/vmware/rules_oss_audit/archive/167dab5b16abdb5996438f22364de544ff24693f.zip", + "patch_args": ["-p1"], + "patches": ["%s:rules_oss_audit_pyyaml.patch" % third_party], }, ] diff --git a/third_party/rules_oss_audit_pyyaml.patch b/third_party/rules_oss_audit_pyyaml.patch new file mode 100644 index 000000000..d2297f018 --- /dev/null +++ b/third_party/rules_oss_audit_pyyaml.patch @@ -0,0 +1,7 @@ +diff --git a/oss_audit/tools/requirements.txt b/oss_audit/tools/requirements.txt +index 932bd69..be2b74d 100644 +--- a/oss_audit/tools/requirements.txt ++++ b/oss_audit/tools/requirements.txt +@@ -1 +1 @@ +-PyYAML==5.4.1 ++PyYAML==6.0.1 From f9ef75aa437e39ab5d93c309d21f7229294eb748 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Tue, 17 Oct 2023 01:11:25 -0400 Subject: [PATCH 21/68] Prevent healthStatusManager NPE on start failure --- src/main/java/build/buildfarm/server/BuildFarmServer.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/build/buildfarm/server/BuildFarmServer.java b/src/main/java/build/buildfarm/server/BuildFarmServer.java index c44896320..d963454de 100644 --- a/src/main/java/build/buildfarm/server/BuildFarmServer.java +++ b/src/main/java/build/buildfarm/server/BuildFarmServer.java @@ -195,8 +195,10 @@ synchronized void stop() throws InterruptedException { private void shutdown() throws InterruptedException { log.info("*** shutting down gRPC server since JVM is shutting down"); prepareServerForGracefulShutdown(); - healthStatusManager.setStatus( - HealthStatusManager.SERVICE_NAME_ALL_SERVICES, ServingStatus.NOT_SERVING); + if (healthStatusManager != null) { + healthStatusManager.setStatus( + HealthStatusManager.SERVICE_NAME_ALL_SERVICES, ServingStatus.NOT_SERVING); + } PrometheusPublisher.stopHttpServer(); healthCheckMetric.labels("stop").inc(); try { From 20512f6980a2bbefbbd87c39209e345e2781934b Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 1 Nov 2023 16:48:04 -0400 Subject: [PATCH 22/68] Consistent check for publicName presence --- .../java/build/buildfarm/common/config/BuildfarmConfigs.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java index d6ebbe9e1..945695228 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java @@ -72,7 +72,7 @@ public static BuildfarmConfigs loadServerConfigs(String[] args) throws Configura log.severe("Could not parse yml configuration file." + e); throw new RuntimeException(e); } - if (!options.publicName.isEmpty()) { + if (!Strings.isNullOrEmpty(options.publicName)) { buildfarmConfigs.getServer().setPublicName(options.publicName); } if (options.port > 0) { From 654032e17eb50f75c84d451b2a80c2f94f2c903b Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 1 Nov 2023 16:49:19 -0400 Subject: [PATCH 23/68] Read through external with query THROUGH=true Specifying a correlated invocation id with a uri containing a THROUGH=true query param will cause the CFC to read a blob through an external input stream, populating locally along the way. This permits client-based replication of blobs, and can enable N+1 replication and traffic balancing for reads. --- src/main/java/build/buildfarm/cas/BUILD | 1 + .../build/buildfarm/cas/cfc/CASFileCache.java | 38 ++++++++++++++++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/cas/BUILD b/src/main/java/build/buildfarm/cas/BUILD index 146206cae..43e2daccd 100644 --- a/src/main/java/build/buildfarm/cas/BUILD +++ b/src/main/java/build/buildfarm/cas/BUILD @@ -27,6 +27,7 @@ java_library( "@maven//:io_grpc_grpc_core", "@maven//:io_grpc_grpc_protobuf", "@maven//:io_grpc_grpc_stub", + "@maven//:io_netty_netty_codec_http", "@maven//:io_prometheus_simpleclient", "@maven//:net_jcip_jcip_annotations", "@maven//:org_projectlombok_lombok", diff --git a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java index 56e53c3b2..d6b47490d 100644 --- a/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java +++ b/src/main/java/build/buildfarm/cas/cfc/CASFileCache.java @@ -92,6 +92,7 @@ import io.grpc.StatusException; import io.grpc.StatusRuntimeException; import io.grpc.stub.ServerCallStreamObserver; +import io.netty.handler.codec.http.QueryStringDecoder; import io.prometheus.client.Counter; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; @@ -99,6 +100,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.URI; +import java.net.URISyntaxException; import java.nio.channels.ClosedByInterruptException; import java.nio.channels.ClosedChannelException; import java.nio.file.FileAlreadyExistsException; @@ -606,6 +609,20 @@ public Blob get(Digest digest) { private static final int CHUNK_SIZE = 128 * 1024; + private static boolean shouldReadThrough(RequestMetadata requestMetadata) { + try { + URI uri = new URI(requestMetadata.getCorrelatedInvocationsId()); + QueryStringDecoder decoder = new QueryStringDecoder(uri); + return decoder + .parameters() + .getOrDefault("THROUGH", ImmutableList.of("false")) + .get(0) + .equals("true"); + } catch (URISyntaxException e) { + return false; + } + } + @Override public void get( Compressor.Value compressor, @@ -614,9 +631,28 @@ public void get( long count, ServerCallStreamObserver blobObserver, RequestMetadata requestMetadata) { + boolean readThrough = shouldReadThrough(requestMetadata); InputStream in; try { - in = newInput(compressor, digest, offset); + if (readThrough && !contains(digest, /* result=*/ null)) { + // really need to be able to reuse/restart the same write over + // multiple requests - if we get successive read throughs for a single + // digest, we should pick up from where we were last time + // Also servers should affinitize + // And share data, so that they can pick the same worker to pull from + // if possible. + Write write = getWrite(compressor, digest, UUID.randomUUID(), requestMetadata); + blobObserver.setOnCancelHandler(write::reset); + in = + new ReadThroughInputStream( + newExternalInput(compressor, digest, 0), + localOffset -> newTransparentInput(compressor, digest, localOffset), + digest.getSizeBytes(), + offset, + write); + } else { + in = newInput(compressor, digest, offset); + } } catch (IOException e) { blobObserver.onError(e); return; From b4359c5af432bf7302080767815e4ce77a1af8c3 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 1 Nov 2023 16:52:41 -0400 Subject: [PATCH 24/68] Add --port option to worker Option to run the worker with a cmdline specification for its gRPC server port. --- .../java/build/buildfarm/common/config/BuildfarmConfigs.java | 3 +++ .../java/build/buildfarm/common/config/BuildfarmOptions.java | 3 +++ src/main/java/build/buildfarm/common/config/ServerOptions.java | 3 --- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java index 945695228..13b41732d 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java @@ -100,6 +100,9 @@ public static BuildfarmConfigs loadWorkerConfigs(String[] args) throws Configura if (!Strings.isNullOrEmpty(options.publicName)) { buildfarmConfigs.getWorker().setPublicName(options.publicName); } + if (options.port >= 0) { + buildfarmConfigs.getWorker().setPort(options.port); + } if (options.prometheusPort >= 0) { buildfarmConfigs.setPrometheusPort(options.prometheusPort); } diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java b/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java index 1dbe0c7f9..45d6f4aea 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java @@ -33,4 +33,7 @@ public class BuildfarmOptions extends OptionsBase { help = "URI for Redis connection. Use 'redis://' or 'rediss://' for the scheme", defaultValue = "") public String redisUri; + + @Option(name = "port", help = "Port for the buildfarm service.", defaultValue = "-1") + public int port; } diff --git a/src/main/java/build/buildfarm/common/config/ServerOptions.java b/src/main/java/build/buildfarm/common/config/ServerOptions.java index 35f47d6d1..b151c24c7 100644 --- a/src/main/java/build/buildfarm/common/config/ServerOptions.java +++ b/src/main/java/build/buildfarm/common/config/ServerOptions.java @@ -18,9 +18,6 @@ /** Command-line options definition for example server. */ public class ServerOptions extends BuildfarmOptions { - @Option(name = "port", abbrev = 'p', help = "Port to use.", defaultValue = "-1") - public int port; - @Option(name = "public_name", abbrev = 'n', help = "Name of this server.", defaultValue = "") public String publicName; } From 51958098e039fcdac2b4230ddb63915fa749f5e7 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 1 Nov 2023 16:53:31 -0400 Subject: [PATCH 25/68] Restore worker --root cmdline specification Root cmdline specification has been broken since the config change of v2. --- .../java/build/buildfarm/common/config/BuildfarmConfigs.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java index 13b41732d..05d7915b2 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmConfigs.java @@ -109,6 +109,9 @@ public static BuildfarmConfigs loadWorkerConfigs(String[] args) throws Configura if (!Strings.isNullOrEmpty(options.redisUri)) { buildfarmConfigs.getBackplane().setRedisUri(options.redisUri); } + if (!Strings.isNullOrEmpty(options.root)) { + buildfarmConfigs.getWorker().setRoot(options.root); + } adjustWorkerConfigs(buildfarmConfigs); return buildfarmConfigs; } From 938c789dcdd8603eb6bb1d0525a8d891dbb077c0 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 1 Nov 2023 16:54:59 -0400 Subject: [PATCH 26/68] Make bf-executor small blob names consistent Remove the size identification for small blobs when uploading with bf-executor. --- src/main/java/build/buildfarm/tools/Executor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/tools/Executor.java b/src/main/java/build/buildfarm/tools/Executor.java index 4f2f9f3d6..901ff33a9 100644 --- a/src/main/java/build/buildfarm/tools/Executor.java +++ b/src/main/java/build/buildfarm/tools/Executor.java @@ -238,7 +238,7 @@ private static void loadFilesIntoCAS(String instanceName, Channel channel, Path ByteStreamStub bsStub = ByteStreamGrpc.newStub(channel); for (Digest missingDigest : missingDigests) { - Path path = blobsDir.resolve(missingDigest.getHash() + "_" + missingDigest.getSizeBytes()); + Path path = blobsDir.resolve(missingDigest.getHash()); if (missingDigest.getSizeBytes() < Size.mbToBytes(1)) { Request request = Request.newBuilder() From 87face12f1544cbd28d9f6e0dc049f7402ed9363 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 2 Nov 2023 12:22:18 -0400 Subject: [PATCH 27/68] Configured output size operation failure Permit installations to control the failure process for operations which produce outputs larger than the maxEntrySizeBytes. A default value of false retains the existing behavior which appears transient and blacklists the executed action key. When enabled, the action will fail under an invalid violation that indicates user error. --- _site/docs/configuration/configuration.md | 1 + examples/config.yml | 1 + .../build/buildfarm/common/config/Worker.java | 2 ++ .../worker/shard/ShardWorkerContext.java | 20 ++++++++++++++----- .../build/buildfarm/worker/shard/Worker.java | 1 + .../worker/shard/ShardWorkerContextTest.java | 1 + 6 files changed, 21 insertions(+), 5 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index 466375373..373968f06 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -260,6 +260,7 @@ backplane: | onlyMulticoreTests | boolean, _false_ | | Only permit tests to exceed the default coresvalue for their min/max-cores range specification (only works with non-zero defaultMaxCores) | | allowBringYourOwnContainer | boolean, _false_ | | Enable execution in a custom Docker container | | errorOperationRemainingResources | boolean, _false_ | | | +| errorOperationOutputSizeExceeded | boolean, _false_ | | Operations which produce single output files which exceed maxEntrySizeBytes will fail with a violation type which implies a user error. When disabled, the violation will indicate a transient error, with the action blacklisted. | | realInputDirectories | List of Strings, _external_ | | A list of paths that will not be subject to the effects of linkInputDirectories setting, may also be used to provide writable directories as input roots for actions which expect to be able to write to an input location and will fail if they cannot | | gracefulShutdownSeconds | Integer, 0 | | Time in seconds to allow for operations in flight to finish when shutdown signal is received | | createSymlinkOutputs | boolean, _false_ | | Creates SymlinkNodes for symbolic links discovered in output paths for actions. No verification of the symlink target path occurs. Buildstream, for example, requires this. | diff --git a/examples/config.yml b/examples/config.yml index 008bf2e1c..69536c5af 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -126,6 +126,7 @@ worker: onlyMulticoreTests: false allowBringYourOwnContainer: false errorOperationRemainingResources: false + errorOperationOutputSizeExceeded: false gracefulShutdownSeconds: 0 sandboxSettings: alwaysUse: false diff --git a/src/main/java/build/buildfarm/common/config/Worker.java b/src/main/java/build/buildfarm/common/config/Worker.java index c446134fe..1a95d9495 100644 --- a/src/main/java/build/buildfarm/common/config/Worker.java +++ b/src/main/java/build/buildfarm/common/config/Worker.java @@ -46,6 +46,8 @@ public class Worker { // If you want GPU actions to run exclusively, define a single GPU resource. private List resources = new ArrayList<>(); + private boolean errorOperationOutputSizeExceeded = false; + public ExecutionPolicy[] getExecutionPolicies() { if (executionPolicies != null) { return executionPolicies; diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index 98474b439..edf7162a3 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -133,6 +133,7 @@ class ShardWorkerContext implements WorkerContext { private final CasWriter writer; private final boolean errorOperationRemainingResources; private final LocalResourceSet resourceSet; + private final boolean errorOperationOutputSizeExceeded; static SetMultimap getMatchProvisions( Iterable policies, int executeStageWidth) { @@ -166,6 +167,7 @@ static SetMultimap getMatchProvisions( boolean onlyMulticoreTests, boolean allowBringYourOwnContainer, boolean errorOperationRemainingResources, + boolean errorOperationOutputSizeExceeded, LocalResourceSet resourceSet, CasWriter writer) { this.name = name; @@ -187,6 +189,7 @@ static SetMultimap getMatchProvisions( this.onlyMulticoreTests = onlyMulticoreTests; this.allowBringYourOwnContainer = allowBringYourOwnContainer; this.errorOperationRemainingResources = errorOperationRemainingResources; + this.errorOperationOutputSizeExceeded = errorOperationOutputSizeExceeded; this.resourceSet = resourceSet; this.writer = writer; } @@ -504,6 +507,7 @@ private void uploadOutputFile( ActionResult.Builder resultBuilder, Path outputPath, Path actionRoot, + String entrySizeViolationType, PreconditionFailure.Builder preconditionFailure) throws IOException, InterruptedException { String outputFile = actionRoot.relativize(outputPath).toString(); @@ -534,7 +538,7 @@ private void uploadOutputFile( outputPath, size, maxEntrySize); preconditionFailure .addViolationsBuilder() - .setType(VIOLATION_TYPE_MISSING) + .setType(entrySizeViolationType) .setSubject(outputFile + ": " + size) .setDescription(message); return; @@ -562,7 +566,7 @@ private void uploadOutputFile( } catch (EntryLimitException e) { preconditionFailure .addViolationsBuilder() - .setType(VIOLATION_TYPE_MISSING) + .setType(entrySizeViolationType) .setSubject("blobs/" + DigestUtil.toString(digest)) .setDescription( "An output could not be uploaded because it exceeded the maximum size of an entry"); @@ -603,6 +607,7 @@ private void uploadOutputDirectory( ActionResult.Builder resultBuilder, Path outputDirPath, Path actionRoot, + String entrySizeViolationType, PreconditionFailure.Builder preconditionFailure) throws IOException, InterruptedException { String outputDir = actionRoot.relativize(outputDirPath).toString(); @@ -683,7 +688,7 @@ private void visitRegularFile(Path file, BasicFileAttributes attrs) throws IOExc } catch (EntryLimitException e) { preconditionFailure .addViolationsBuilder() - .setType(VIOLATION_TYPE_MISSING) + .setType(entrySizeViolationType) .setSubject("blobs/" + DigestUtil.toString(digest)) .setDescription( "An output could not be uploaded because it exceeded the maximum size of an entry"); @@ -730,14 +735,19 @@ public FileVisitResult postVisitDirectory(Path dir, IOException exc) { public void uploadOutputs( Digest actionDigest, ActionResult.Builder resultBuilder, Path actionRoot, Command command) throws IOException, InterruptedException, StatusException { + String entrySizeViolationType = + errorOperationOutputSizeExceeded ? VIOLATION_TYPE_INVALID : VIOLATION_TYPE_MISSING; + PreconditionFailure.Builder preconditionFailure = PreconditionFailure.newBuilder(); List outputPaths = CommandUtils.getResolvedOutputPaths(command, actionRoot); for (Path outputPath : outputPaths) { if (Files.isDirectory(outputPath)) { - uploadOutputDirectory(resultBuilder, outputPath, actionRoot, preconditionFailure); + uploadOutputDirectory( + resultBuilder, outputPath, actionRoot, entrySizeViolationType, preconditionFailure); } else { - uploadOutputFile(resultBuilder, outputPath, actionRoot, preconditionFailure); + uploadOutputFile( + resultBuilder, outputPath, actionRoot, entrySizeViolationType, preconditionFailure); } } checkPreconditionFailure(actionDigest, preconditionFailure.build()); diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index ce69eb3d1..be0c24a1d 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -611,6 +611,7 @@ public void start() throws ConfigurationException, InterruptedException, IOExcep configs.getWorker().isOnlyMulticoreTests(), configs.getWorker().isAllowBringYourOwnContainer(), configs.getWorker().isErrorOperationRemainingResources(), + configs.getWorker().isErrorOperationOutputSizeExceeded(), LocalResourceSetUtils.create(configs.getWorker().getResources()), writer); diff --git a/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java b/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java index c24e681ab..efddbdb86 100644 --- a/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java +++ b/src/test/java/build/buildfarm/worker/shard/ShardWorkerContextTest.java @@ -106,6 +106,7 @@ WorkerContext createTestContext(Iterable policies) { /* onlyMulticoreTests=*/ false, /* allowBringYourOwnContainer=*/ false, /* errorOperationRemainingResources=*/ false, + /* errorOperationOutputSizeExceeded=*/ false, /* resourceSet=*/ new LocalResourceSet(), writer); } From 58faec9eac95a2e610831051aff507bfb3f81110 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 2 Nov 2023 12:03:07 -0400 Subject: [PATCH 28/68] Restore abbrev port as -p --- .../build/buildfarm/common/config/BuildfarmOptions.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java b/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java index 45d6f4aea..9fc3f1ed1 100644 --- a/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java +++ b/src/main/java/build/buildfarm/common/config/BuildfarmOptions.java @@ -34,6 +34,10 @@ public class BuildfarmOptions extends OptionsBase { defaultValue = "") public String redisUri; - @Option(name = "port", help = "Port for the buildfarm service.", defaultValue = "-1") + @Option( + name = "port", + abbrev = 'p', + help = "Port for the buildfarm service.", + defaultValue = "-1") public int port; } From cf6fc58d851f529da103129e2a4b36eb6af748d0 Mon Sep 17 00:00:00 2001 From: Jerry Marino Date: Tue, 31 Oct 2023 13:04:16 -0700 Subject: [PATCH 29/68] Update zstd-jni for latest version There's been a few releases of it by now and this pulls the latest. For buildfarm, notable changes included performance enhancments during decompression. See: https://github.com/facebook/zstd/releases/tag/v1.5.5 --- defs.bzl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/defs.bzl b/defs.bzl index 4b0a5e49b..147177835 100644 --- a/defs.bzl +++ b/defs.bzl @@ -68,7 +68,7 @@ def buildfarm_init(name = "buildfarm"): "com.github.jnr:jnr-posix:3.0.53", "com.github.pcj:google-options:1.0.0", "com.github.serceman:jnr-fuse:0.5.5", - "com.github.luben:zstd-jni:1.5.2-1", + "com.github.luben:zstd-jni:1.5.5-7", "com.github.oshi:oshi-core:6.4.0", "com.google.auth:google-auth-library-credentials:0.9.1", "com.google.auth:google-auth-library-oauth2-http:0.9.1", From 6bc70e1b3961c74b93886408b57f77bf1689e0f8 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Fri, 3 Nov 2023 11:28:43 -0400 Subject: [PATCH 30/68] Attempt to resolve windows stamping --- .bazelci/presubmit.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.bazelci/presubmit.yml b/.bazelci/presubmit.yml index 98b121e21..728e68d02 100644 --- a/.bazelci/presubmit.yml +++ b/.bazelci/presubmit.yml @@ -70,6 +70,7 @@ tasks: build_targets: - "..." test_flags: + - "--@rules_jvm_external//settings:stamp_manifest=False" - "--test_tag_filters=-integration,-redis" test_targets: - "..." From b2267253bbf16ab30fe5b9d1b2dfaa8c87c295ae Mon Sep 17 00:00:00 2001 From: Anshuman Mishra Date: Wed, 1 Nov 2023 19:53:01 -0700 Subject: [PATCH 31/68] Bug: Fix workerSet update logic for RemoteCasWriter --- .../worker/shard/RemoteCasWriter.java | 32 +++++++++---------- .../build/buildfarm/worker/shard/Worker.java | 2 +- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/shard/RemoteCasWriter.java b/src/main/java/build/buildfarm/worker/shard/RemoteCasWriter.java index 8a5bfdb52..b03c2794e 100644 --- a/src/main/java/build/buildfarm/worker/shard/RemoteCasWriter.java +++ b/src/main/java/build/buildfarm/worker/shard/RemoteCasWriter.java @@ -21,6 +21,7 @@ import build.bazel.remote.execution.v2.Compressor; import build.bazel.remote.execution.v2.Digest; import build.bazel.remote.execution.v2.RequestMetadata; +import build.buildfarm.backplane.Backplane; import build.buildfarm.common.Size; import build.buildfarm.common.Write; import build.buildfarm.common.grpc.Retrier; @@ -48,13 +49,13 @@ @Log public class RemoteCasWriter implements CasWriter { - private final Set workerSet; + private final Backplane backplane; private final LoadingCache workerStubs; private final Retrier retrier; public RemoteCasWriter( - Set workerSet, LoadingCache workerStubs, Retrier retrier) { - this.workerSet = workerSet; + Backplane backplane, LoadingCache workerStubs, Retrier retrier) { + this.backplane = backplane; this.workerStubs = workerStubs; this.retrier = retrier; } @@ -114,20 +115,19 @@ public void insertBlob(Digest digest, ByteString content) } private String getRandomWorker() throws IOException { - synchronized (workerSet) { - if (workerSet.isEmpty()) { - throw new IOException("no available workers"); - } - Random rand = new Random(); - int index = rand.nextInt(workerSet.size()); - // best case no allocation average n / 2 selection - Iterator iter = workerSet.iterator(); - String worker = null; - while (iter.hasNext() && index-- >= 0) { - worker = iter.next(); - } - return worker; + Set workerSet = backplane.getStorageWorkers(); + if (workerSet.isEmpty()) { + throw new IOException("no available workers"); } + Random rand = new Random(); + int index = rand.nextInt(workerSet.size()); + // best case no allocation average n / 2 selection + Iterator iter = workerSet.iterator(); + String worker = null; + while (iter.hasNext() && index-- >= 0) { + worker = iter.next(); + } + return worker; } private Instance workerStub(String worker) { diff --git a/src/main/java/build/buildfarm/worker/shard/Worker.java b/src/main/java/build/buildfarm/worker/shard/Worker.java index be0c24a1d..ce7a3e738 100644 --- a/src/main/java/build/buildfarm/worker/shard/Worker.java +++ b/src/main/java/build/buildfarm/worker/shard/Worker.java @@ -584,7 +584,7 @@ public void start() throws ConfigurationException, InterruptedException, IOExcep CasWriter writer; if (!configs.getWorker().getCapabilities().isCas()) { Retrier retrier = new Retrier(Backoff.sequential(5), Retrier.DEFAULT_IS_RETRIABLE); - writer = new RemoteCasWriter(backplane.getStorageWorkers(), workerStubs, retrier); + writer = new RemoteCasWriter(backplane, workerStubs, retrier); } else { writer = new LocalCasWriter(execFileSystem); } From 751ac90e9d18c8a25c3e8d8c26c3f4b4457310bc Mon Sep 17 00:00:00 2001 From: George Gensure Date: Fri, 3 Nov 2023 20:15:43 -0400 Subject: [PATCH 32/68] Detail storage requirements Update for further docs related to storage+type functionality Remove outdated Operation Queue worker definitions --- _site/docs/architecture/workers.md | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/_site/docs/architecture/workers.md b/_site/docs/architecture/workers.md index 28765b6a7..ee3e67f8a 100644 --- a/_site/docs/architecture/workers.md +++ b/_site/docs/architecture/workers.md @@ -7,20 +7,21 @@ nav_order: 2 # Workers -Workers of all types throughout buildfarm are responsible for presenting execution roots to operations that they are matched with, fetching content from a CAS, executing those processes, and reporting the outputs and results of executions. Additionally, buildfarm supports some common behaviors across worker types: +Workers have two major roles in Buildfarm: Execution and CAS Shard. Either of these options can be disabled, though a worker with both disabled provides no value. -* ExecutionPolicies, which allow for explicit and implicit behaviors to control execution. -* A CAS FileCache, which is capable of reading through content for Digests of files or directories, and efficiently presenting those contents based on usage and reference counting, as well as support for cascading into delegate CASs. -* Concurrent pipelined execution of operations, with support for superscalar stages at input fetch and execution. -* Operation exclusivity, preventing the same operation from running through the worker pipeline concurrently. +Regardless of role, a worker must have a local FILESYSTEM type [storage](https://bazelbuild.github.io/bazel-buildfarm/docs/configuration/configuration/#worker-cas) to retain content. This storage serves both as a resident LRU cache for Execution I/O, and the local storage for a CAS Shard. Workers can delegate to successive storage declarations (FILESYSTEM or GRPC), with read-through or expiration waterfall if configured, but only the first storage entry will be used for Executions. -# Worker Types +## Execution -## Operation Queue +Execution Workers are responsible for matching their environments against operations, presenting execution roots to those operations, fetching content from a CAS, executing processes required to complete the operations, and reporting the outputs and results of executions. Control and delivery of these behaviors is accomplished with several mechanisms: -Operation Queue workers are responsible for taking operations from the Memory OperationQueue service and reporting their contents via external CAS and AC services. Executions are the only driving force for their CAS FileCache. For more details on configuring the operation queue, [see here](https://github.com/bazelbuild/bazel-buildfarm/wiki/Operation-Queue). +* A CAS FileCache, which is capable of reading through content for Digests of files or directories, and efficiently presenting those contents based on usage and reference counting, as well as support for cascading into delegate CASs. +* ExecutionPolicies, which allow for explicit and implicit behaviors to control execution. +* Execution Resources to limit concurrent execution in installation-defined resource traunches. +* Concurrent pipelined execution of operations, with support for superscalar stages at input fetch and execution. +* Operation exclusivity, preventing the same operation from running through the worker pipeline concurrently. -## Shard +## CAS Shard Sharded workers interact with the shard backplane for both execution and CAS presentation. Their CAS FileCache serves a CAS gRPC interface as well as the execution root factory. @@ -56,18 +57,20 @@ The Report Result stage injects any outputs from the operation into the CAS, and # Exec Filesystem -Workers must present Exec Filesystems for actions, and manage their existence for the lifetime of an operation's presence within the pipeline. The realization of an operation's execution root with the execution filesystem constitutes a transaction that the operating directory for an action will appear, be writable for outputs, and released and be made unavailable as it proceeds and exits the pipeline. +Workers use ExecFileSystems to present content to actions, and manage their existence for the lifetime of an operation's presence within the pipeline. The realization of an operation's execution root with the execution filesystem constitutes a transaction that the operating directory for an action will appear, be writable for outputs, and released and be made unavailable as it proceeds and exits the pipeline. This means that an action's entire input directory must be available on a filesystem from a unique location per operation - the _Operation Action Input Root_, or just _Root_. Each input file within the Root must contain the content of the inputs, its requested executability via FileNode, and each directory must contain at the outset, child input files and directories. The filesystem is free to handle unspecified outputs as it sees fit, but the directory hierarchy of output files from the Root must be created before execution, and writable during it. When execution and observation of the outputs is completed, the exec filesystem will be asked to destroy the Root and release any associated resources from its retention. -There are two implementations of Execution Filesystem in Buildfarm. Choosing either a `filesystem` or `fuse` `cas` type in the worker config as the first `cas` entry will choose the _CASFileCache_ or _FuseCAS_ implementations, respectively. +Choosing a `filesystem` `storage` type in the worker config as the first `storage` entry will select the _CASFileCache_ _CFCExecFileSystem_. Choosing any other `storage` type will create a _FuseCAS_ _FuseExecFilesystem_. + +***We strongly recommend the use of `filesystem` `storage` as the ExecFileSystem-selecting `storage` entry, the _FuseCAS_ is experimental and may not function reliably over long hauls/with substantial load*** ## CASFileCache/CFCExecFilesystem The CASFileCache provides an Exec Filesystem via CFCExecFilesystem. The (CASFileCache)'s retention of paths is used to reflect individual files, with these paths hard-linked in CFCExecFilesystem under representative directories of the input root to signify usage. The CASFileCache directory retention system is also used to provide a configurable utilization of entire directory trees as a symlink, which was a heuristic optimization applied when substantial cost was observed setting up static trees of input links for operations compared to their execution time. `link_input_directories` in the common Worker configuration will enable this heuristic. Outputs of actions are physically streamed into CAS writes when they are observed after an action execution. -The CASFileCache's persistence in the filesystem and the availability of common POSIX features like symlinks and inode-based reference counts on almost any filesystem implementation have made it a solid choice for extremely large CAS installations - it scales to multi-TB host attached storages with millions of entries with relative ease. +The CASFileCache's persistence in the filesystem and the availability of common POSIX features like symlinks and inode-based reference counts on almost any filesystem implementation have made it a solid choice for extremely large CAS installations - it scales to multi-TB host attached storages containing millions of entries with relative ease. There are plans to improve CASFileCache that will be reflected in improved performance and memory footprint for the features used by CFCExecFilesystem. @@ -75,4 +78,4 @@ There are plans to improve CASFileCache that will be reflected in improved perfo A fuse implementation to provide Roots exists and is specifiable as well. This was an experiment to discover the capacity of a fuse to represent Roots transparently with a ContentAddressableStorage backing, and has not been fully vetted to provide the same reliability as the CFCExecFilesystem. This system is capable of blinking entire trees into existence with ease, as well as supporting write-throughs for outputs suitable for general purpose execution. Some problems with this type were initially observed and never completely resolved, including guaranteed resource release on Root destruction. This implementation is also only built to be backed by its own Memory CAS, with no general purpose CAS support added due to the difficulty of supporting a transaction model for an input tree to enforce the contract of availability. It remains unoptimized yet functional, but difficulties with integrating libfuse 3 into the bazel build, as well as time constraints, have kept it from being scaled and expanded as the rest of Buildfarm has grown. -There are plans to revisit this implementation and bring it back into viability with a CASFileCache-like backing. \ No newline at end of file +There are plans to revisit this implementation and bring it back into viability with a CASFileCache-like backing. From 2bf3eae0b80c55567352ad4b675ea01db32687f7 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Fri, 3 Nov 2023 20:16:22 -0400 Subject: [PATCH 33/68] Fix worker execution env title --- _site/docs/architecture/worker-execution-environment.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/_site/docs/architecture/worker-execution-environment.md b/_site/docs/architecture/worker-execution-environment.md index 89bee694f..a49343fda 100644 --- a/_site/docs/architecture/worker-execution-environment.md +++ b/_site/docs/architecture/worker-execution-environment.md @@ -1,6 +1,6 @@ --- layout: default -title: Workers +title: Worker Execution Environment parent: Architecture nav_order: 3 --- @@ -124,4 +124,4 @@ java_image( And now that this is in place, we can use the following to build the container and make it available to our local docker daemon: -`bazel run :buildfarm-shard-worker-ubuntu20-java14` \ No newline at end of file +`bazel run :buildfarm-shard-worker-ubuntu20-java14` From b6bddffdeabf108e93e4224829ae9f7c21ba4345 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Fri, 3 Nov 2023 21:32:47 -0400 Subject: [PATCH 34/68] Add storage example descriptions --- _site/docs/configuration/configuration.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index 373968f06..83d56a590 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -325,6 +325,8 @@ worker: ### Worker CAS +Unless specified, options are only relevant for FILESYSTEM type + | Configuration | Accepted and _Default_ Values | Description | |------------------------------|-------------------------------|---------------------------------------------------------------------------------------------------------------| | type | _FILESYSTEM_, GRPC | Type of CAS used | @@ -336,6 +338,8 @@ worker: Example: +This definition will create a filesystem-based CAS file cache at the path "/cache" on the worker that will reject entries over 2GiB in size, and will expire LRU blobs when the aggregate size of all blobs exceeds 2GiB in order to insert additional entries. + ```yaml worker: storages: @@ -343,14 +347,15 @@ worker: path: "cache" maxSizeBytes: 2147483648 # 2 * 1024 * 1024 * 1024 maxEntrySizeBytes: 2147483648 # 2 * 1024 * 1024 * 1024 - target: ``` +This definition elides FILESYSTEM configuration with '...', will read-through an external GRPC CAS supporting the REAPI CAS Services into its storage, and will attempt to write expiring entries into the GRPC CAS (i.e. pushing new entries into the head of a worker LRU list will drop the entries from the tail into the GRPC CAS). + ``` worker: storages: - type: FILESYSTEM - path: "cache" + ... - type: GRPC target: "cas.external.com:1234" ``` From c49092568934d67dd5d80952e428b52757582992 Mon Sep 17 00:00:00 2001 From: Justin Won Date: Mon, 6 Nov 2023 08:10:36 -0800 Subject: [PATCH 35/68] Check for context cancelled before responding to error (#1526) When a write fails because the write was already cancelled before due to something like deadline exceeded, we get an unknown error. The exception comes from here and when it gets to errorResponse(), it only checks if status code is cancelled. In this case the status code is unknown, so we need to check if context is cancelled to prevent responseObserver from being invoked The code change adds checking if context is cancelled and a unit test testing when the exception has context cancelled. --- .../common/services/WriteStreamObserver.java | 3 +- .../services/WriteStreamObserverTest.java | 48 +++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/common/services/WriteStreamObserver.java b/src/main/java/build/buildfarm/common/services/WriteStreamObserver.java index 36cc0595a..dd52f7769 100644 --- a/src/main/java/build/buildfarm/common/services/WriteStreamObserver.java +++ b/src/main/java/build/buildfarm/common/services/WriteStreamObserver.java @@ -271,7 +271,8 @@ private void logWriteRequest(WriteRequest request, Exception e) { private boolean errorResponse(Throwable t) { if (exception.compareAndSet(null, t)) { - if (Status.fromThrowable(t).getCode() == Status.Code.CANCELLED) { + if (Status.fromThrowable(t).getCode() == Status.Code.CANCELLED + || Context.current().isCancelled()) { return false; } boolean isEntryLimitException = t instanceof EntryLimitException; diff --git a/src/test/java/build/buildfarm/common/services/WriteStreamObserverTest.java b/src/test/java/build/buildfarm/common/services/WriteStreamObserverTest.java index 424922399..b6d144a66 100644 --- a/src/test/java/build/buildfarm/common/services/WriteStreamObserverTest.java +++ b/src/test/java/build/buildfarm/common/services/WriteStreamObserverTest.java @@ -28,6 +28,7 @@ import io.grpc.Context; import io.grpc.Context.CancellableContext; import io.grpc.stub.StreamObserver; +import java.io.IOException; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.junit.Test; @@ -90,4 +91,51 @@ public void cancelledBeforeGetOutputIsSilent() throws Exception { verify(out, times(1)).close(); verifyZeroInteractions(responseObserver); } + + @Test + public void noErrorWhenContextCancelled() throws Exception { + CancellableContext context = Context.current().withCancellation(); + Instance instance = mock(Instance.class); + StreamObserver responseObserver = mock(StreamObserver.class); + ByteString cancelled = ByteString.copyFromUtf8("cancelled data"); + Digest cancelledDigest = DIGEST_UTIL.compute(cancelled); + UUID uuid = UUID.randomUUID(); + UploadBlobRequest uploadBlobRequest = + UploadBlobRequest.newBuilder() + .setBlob(BlobInformation.newBuilder().setDigest(cancelledDigest)) + .setUuid(uuid.toString()) + .build(); + SettableFuture future = SettableFuture.create(); + Write write = mock(Write.class); + when(write.getFuture()).thenReturn(future); + when(write.isComplete()).thenReturn(Boolean.TRUE); + when(instance.getBlobWrite( + eq(Compressor.Value.IDENTITY), + eq(cancelledDigest), + eq(uuid), + any(RequestMetadata.class))) + .thenReturn(write); + + WriteStreamObserver observer = + context.call( + () -> new WriteStreamObserver(instance, 1, SECONDS, () -> {}, responseObserver)); + context.run( + () -> + observer.onNext( + WriteRequest.newBuilder() + .setResourceName(uploadResourceName(uploadBlobRequest)) + .setData(cancelled) + .build())); + context.cancel(new RuntimeException("Cancelled by test")); + future.setException(new IOException("test cancel")); + + verify(write, times(1)).isComplete(); + verify(instance, times(1)) + .getBlobWrite( + eq(Compressor.Value.IDENTITY), + eq(cancelledDigest), + eq(uuid), + any(RequestMetadata.class)); + verifyZeroInteractions(responseObserver); + } } From 2a51c3183c72147c4e1d73ad184941ef04a2ea6e Mon Sep 17 00:00:00 2001 From: Jason Schroeder Date: Mon, 16 Oct 2023 11:41:41 -0700 Subject: [PATCH 36/68] chore(deps): bump com.google.errorprone:error-prone Release notes: https://github.com/google/error-prone/releases/tag/v2.22.0 --- defs.bzl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/defs.bzl b/defs.bzl index 147177835..a40dccaef 100644 --- a/defs.bzl +++ b/defs.bzl @@ -74,8 +74,8 @@ def buildfarm_init(name = "buildfarm"): "com.google.auth:google-auth-library-oauth2-http:0.9.1", "com.google.code.findbugs:jsr305:3.0.1", "com.google.code.gson:gson:2.9.0", - "com.google.errorprone:error_prone_annotations:2.9.0", - "com.google.errorprone:error_prone_core:0.92", + "com.google.errorprone:error_prone_annotations:2.22.0", + "com.google.errorprone:error_prone_core:2.22.0", "com.google.guava:failureaccess:1.0.1", "com.google.guava:guava:32.1.1-jre", "com.google.j2objc:j2objc-annotations:1.1", From 7ea1a9fb1ce46705ee774c98e92fe097f16898f9 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Wed, 8 Nov 2023 15:49:27 -0500 Subject: [PATCH 37/68] Worker name execution properties matching --- .../buildfarm/common/ExecutionProperties.java | 7 +++ .../worker/DequeueMatchEvaluator.java | 23 +++++++++- .../worker/shard/ShardWorkerContext.java | 3 +- .../worker/DequeueMatchEvaluatorTest.java | 44 +++++++++++++------ 4 files changed, 62 insertions(+), 15 deletions(-) diff --git a/src/main/java/build/buildfarm/common/ExecutionProperties.java b/src/main/java/build/buildfarm/common/ExecutionProperties.java index 8126bd607..ee3975d6d 100644 --- a/src/main/java/build/buildfarm/common/ExecutionProperties.java +++ b/src/main/java/build/buildfarm/common/ExecutionProperties.java @@ -293,4 +293,11 @@ public class ExecutionProperties { * operation queue). */ public static final String POOL = "Pool"; + + /** + * @field WORKER + * @brief The exec_property to ensure that the action only runs on the worker name given. + * @details Useful for diagnosing woker issues by targeting builds to a specific worker. + */ + public static final String WORKER = "Worker"; } diff --git a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java index 3536ed6ba..59ccd00e3 100644 --- a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java +++ b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java @@ -46,6 +46,7 @@ public class DequeueMatchEvaluator { * @brief Decide whether the worker should keep the operation or put it back on the queue. * @details Compares the platform properties of the worker to the operation's platform properties. * @param workerProvisions The provisions of the worker. + * @param name Worker name. * @param resourceSet The limited resources that the worker has available. * @param queueEntry An entry recently removed from the queue. * @return Whether or not the worker should accept or reject the queue entry. @@ -56,9 +57,10 @@ public class DequeueMatchEvaluator { @NotNull public static boolean shouldKeepOperation( SetMultimap workerProvisions, + String name, LocalResourceSet resourceSet, QueueEntry queueEntry) { - return shouldKeepViaPlatform(workerProvisions, resourceSet, queueEntry.getPlatform()); + return shouldKeepViaPlatform(workerProvisions, name, resourceSet, queueEntry.getPlatform()); } /** @@ -67,6 +69,7 @@ public static boolean shouldKeepOperation( * @details Compares the platform properties of the worker to the platform properties of the * operation. * @param workerProvisions The provisions of the worker. + * @param name Worker name. * @param resourceSet The limited resources that the worker has available. * @param platform The platforms of operation. * @return Whether or not the worker should accept or reject the operation. @@ -76,6 +79,7 @@ public static boolean shouldKeepOperation( @NotNull private static boolean shouldKeepViaPlatform( SetMultimap workerProvisions, + String name, LocalResourceSet resourceSet, Platform platform) { // attempt to execute everything the worker gets off the queue, @@ -84,6 +88,12 @@ private static boolean shouldKeepViaPlatform( if (!LocalResourceSetUtils.claimResources(platform, resourceSet)) { return false; } + + // The action might be requesting to run on a particular action + if (!keepForThisWorker(platform, name)) { + return false; + } + if (configs.getWorker().getDequeueMatchSettings().isAcceptEverything()) { return true; } @@ -91,6 +101,17 @@ private static boolean shouldKeepViaPlatform( return satisfiesProperties(workerProvisions, platform); } + private static boolean keepForThisWorker(Platform platform, String name) { + for (Platform.Property property : platform.getPropertiesList()) { + if (property.getName().equals(ExecutionProperties.WORKER) + && !property.getValue().equals(name)) { + // requested worker does not match this worker, reject + return false; + } + } + return true; + } + /** * @brief Decide whether the worker should keep the operation by comparing its platform properties * with the queue entry. diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index edf7162a3..519444491 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -315,7 +315,8 @@ private QueueEntry takeEntryOffOperationQueue(MatchListener listener) private void decideWhetherToKeepOperation(QueueEntry queueEntry, MatchListener listener) throws IOException, InterruptedException { if (queueEntry == null - || DequeueMatchEvaluator.shouldKeepOperation(matchProvisions, resourceSet, queueEntry)) { + || DequeueMatchEvaluator.shouldKeepOperation( + matchProvisions, name, resourceSet, queueEntry)) { listener.onEntry(queueEntry); } else { backplane.rejectOperation(queueEntry); diff --git a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java index 47f25cdf7..290470b84 100644 --- a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java +++ b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java @@ -59,7 +59,8 @@ public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -87,7 +88,8 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker accepts because it has more cores than the min-cores requested @@ -117,7 +119,8 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker rejects because it has less cores than the min-cores requested @@ -146,7 +149,8 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker accepts because it has the same cores as the min-cores requested @@ -175,7 +179,8 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT assertThat(shouldKeep).isFalse(); @@ -184,7 +189,9 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -193,7 +200,9 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -224,7 +233,8 @@ public void shouldKeepOperationClaimsResource() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -232,7 +242,9 @@ public void shouldKeepOperationClaimsResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -269,7 +281,8 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -278,7 +291,9 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(2); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -287,7 +302,9 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(0); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -330,7 +347,8 @@ public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + DequeueMatchEvaluator.shouldKeepOperation( + workerProvisions, "worker-name", resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. From a4822c125ae45b775e95d5eacbfa137aa38145ae Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Fri, 21 Apr 2023 15:31:25 -0400 Subject: [PATCH 38/68] updates --- _site/docs/configuration/configuration.md | 12 +++++++----- examples/config.yml | 1 + .../buildfarm/common/config/SandboxSettings.java | 7 +++++++ .../buildfarm/worker/resources/ResourceDecider.java | 8 +++++++- 4 files changed, 22 insertions(+), 6 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index 83d56a590..4be39b912 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -291,11 +291,12 @@ worker: ### Sandbox Settings -| Configuration | Accepted and _Default_ Values | Description | -|---------------|-------------------------------|---------------------------------------------------| -| alwaysUse | boolean, _false_ | Enforce that the sandbox be used on every acion. | -| selectForBlockNetwork | boolean, _false_ | `block-network` enables sandbox action execution. | -| selectForTmpFs | boolean, _false_ | `tmpfs` enables sandbox action execution. | +| Configuration | Accepted and _Default_ Values | Description | +|---------------|-------------------------------|------------------------------------------------------| +| alwaysUse | boolean, _false_ | Enforce that the sandbox be used on every acion. | +| alwaysUseTmpFs | boolean, _false_ | Enforce that the sandbox uses tmpfs on every acion. | +| selectForBlockNetwork | boolean, _false_ | `block-network` enables sandbox action execution. | +| selectForTmpFs | boolean, _false_ | `tmpfs` enables sandbox action execution. | Example: @@ -303,6 +304,7 @@ Example: worker: sandboxSettings: alwaysUse: true + alwaysUseTmpFs: true selectForBlockNetwork: false selectForTmpFs: false ``` diff --git a/examples/config.yml b/examples/config.yml index 69536c5af..b0626f467 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -130,6 +130,7 @@ worker: gracefulShutdownSeconds: 0 sandboxSettings: alwaysUse: false + alwaysUseTmpFs: false selectForBlockNetwork: false selectForTmpFs: false createSymlinkOutputs: false diff --git a/src/main/java/build/buildfarm/common/config/SandboxSettings.java b/src/main/java/build/buildfarm/common/config/SandboxSettings.java index 030377bf0..b7e238a07 100644 --- a/src/main/java/build/buildfarm/common/config/SandboxSettings.java +++ b/src/main/java/build/buildfarm/common/config/SandboxSettings.java @@ -32,6 +32,13 @@ public class SandboxSettings { */ public boolean alwaysUse = false; + /** + * @field alwaysUseTmpFs + * @brief Whether or not to always use tmpfs when using the sandbox. + * @details It may be preferred to enforce sandbox usage than rely on client selection. + */ + public boolean alwaysUseTmpFs = false; + /** * @field selectForBlockNetwork * @brief If the action requires "block network" use the sandbox to fulfill this request. diff --git a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java index a5229a337..bd352a00a 100644 --- a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java +++ b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java @@ -203,10 +203,16 @@ private static void decideSandboxUsage(ResourceLimits limits, SandboxSettings sa // configured on if (sandbox.isAlwaysUse()) { limits.useLinuxSandbox = true; - limits.description.add("enabled"); + limits.description.add("enabled sandbox by default"); return; } + // configured on + if (sandbox.isAlwaysUseTmpFs()) { + limits.tmpFs = true; + limits.description.add("enabled tmpfs by default"); + } + // selected based on other features if (limits.network.blockNetwork && sandbox.isSelectForBlockNetwork()) { limits.useLinuxSandbox = true; From f1ea9b51280557b3b6b68c8a36fc53ecb35c9bc8 Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Fri, 21 Apr 2023 16:45:31 -0400 Subject: [PATCH 39/68] updates --- .../common/config/SandboxSettings.java | 7 ++++ .../worker/resources/ResourceDecider.java | 11 ++++-- .../worker/resources/ResourceLimits.java | 7 ++++ .../worker/shard/ShardWorkerContext.java | 39 ++++++++++--------- 4 files changed, 42 insertions(+), 22 deletions(-) diff --git a/src/main/java/build/buildfarm/common/config/SandboxSettings.java b/src/main/java/build/buildfarm/common/config/SandboxSettings.java index b7e238a07..d9988c845 100644 --- a/src/main/java/build/buildfarm/common/config/SandboxSettings.java +++ b/src/main/java/build/buildfarm/common/config/SandboxSettings.java @@ -32,6 +32,13 @@ public class SandboxSettings { */ public boolean alwaysUse = false; + /** + * @field alwaysUseCgroups + * @brief Whether or not to use cgroups when sandboxing actions. + * @details It may be preferred to enforce cgroup usage. + */ + public boolean alwaysUseCgroups = false; + /** * @field alwaysUseTmpFs * @brief Whether or not to always use tmpfs when using the sandbox. diff --git a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java index bd352a00a..78021f1ed 100644 --- a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java +++ b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java @@ -200,14 +200,17 @@ private static void adjustLimits( } private static void decideSandboxUsage(ResourceLimits limits, SandboxSettings sandbox) { - // configured on + + // Decide which sandbox limitations are enabled by default acording to the deployment's + // configuration. if (sandbox.isAlwaysUse()) { limits.useLinuxSandbox = true; limits.description.add("enabled sandbox by default"); - return; } - - // configured on + if (sandbox.isAlwaysUseCgroups()) { + limits.cgroups = true; + limits.description.add("enabled cgroups by default"); + } if (sandbox.isAlwaysUseTmpFs()) { limits.tmpFs = true; limits.description.add("enabled tmpfs by default"); diff --git a/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java b/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java index 4e5601625..e3ddc4485 100644 --- a/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java +++ b/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java @@ -75,6 +75,13 @@ public class ResourceLimits { */ public ContainerSettings containerSettings = new ContainerSettings(); + /** + * @field cgroups + * @brief Whether to use cgroups for resource limitation. + * @details Decides whether to use cgroups for restricting cores, mem, etc. + */ + public boolean cgroups = true; + /** * @field cpu * @brief Resource limitations on CPUs. diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index 519444491..590d492de 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -922,28 +922,31 @@ IOResource limitSpecifiedExecution( // ResourceLimits object. We apply the cgroup settings to file resources // and collect group names to use on the CLI. String operationId = getOperationId(operationName); - final Group group = operationsGroup.getChild(operationId); ArrayList resources = new ArrayList<>(); - ArrayList usedGroups = new ArrayList<>(); - // Possibly set core restrictions. - if (limits.cpu.limit) { - applyCpuLimits(group, limits, resources); - usedGroups.add(group.getCpu().getName()); - } + if (limits.cgroups) { + final Group group = operationsGroup.getChild(operationId); + ArrayList usedGroups = new ArrayList<>(); - // Possibly set memory restrictions. - if (limits.mem.limit) { - applyMemLimits(group, limits, resources); - usedGroups.add(group.getMem().getName()); - } + // Possibly set core restrictions. + if (limits.cpu.limit) { + applyCpuLimits(group, limits, resources); + usedGroups.add(group.getCpu().getName()); + } - // Decide the CLI for running under cgroups - if (!usedGroups.isEmpty()) { - arguments.add( - configs.getExecutionWrappers().getCgroups(), - "-g", - String.join(",", usedGroups) + ":" + group.getHierarchy()); + // Possibly set memory restrictions. + if (limits.mem.limit) { + applyMemLimits(group, limits, resources); + usedGroups.add(group.getMem().getName()); + } + + // Decide the CLI for running under cgroups + if (!usedGroups.isEmpty()) { + arguments.add( + configs.getExecutionWrappers().getCgroups(), + "-g", + String.join(",", usedGroups) + ":" + group.getHierarchy()); + } } // Possibly set network restrictions. From ccf763da22b173967b9ee8bad1dcc4f3b8f80a1e Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Fri, 21 Apr 2023 16:49:24 -0400 Subject: [PATCH 40/68] updates --- .../java/build/buildfarm/common/config/SandboxSettings.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/build/buildfarm/common/config/SandboxSettings.java b/src/main/java/build/buildfarm/common/config/SandboxSettings.java index d9988c845..fc56aa889 100644 --- a/src/main/java/build/buildfarm/common/config/SandboxSettings.java +++ b/src/main/java/build/buildfarm/common/config/SandboxSettings.java @@ -37,7 +37,7 @@ public class SandboxSettings { * @brief Whether or not to use cgroups when sandboxing actions. * @details It may be preferred to enforce cgroup usage. */ - public boolean alwaysUseCgroups = false; + public boolean alwaysUseCgroups = true; /** * @field alwaysUseTmpFs From b7f56613d1aa5a963beb254028c1ea8e86acb17c Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Fri, 21 Apr 2023 16:56:00 -0400 Subject: [PATCH 41/68] updates --- _site/docs/configuration/configuration.md | 8 ++++++-- examples/config.yml | 3 ++- .../build/buildfarm/common/config/SandboxSettings.java | 4 ++-- .../build/buildfarm/worker/resources/ResourceDecider.java | 3 +-- .../buildfarm/worker/resources/ResourceDeciderTest.java | 2 +- 5 files changed, 12 insertions(+), 8 deletions(-) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index 4be39b912..83d4dcc8d 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -293,7 +293,8 @@ worker: | Configuration | Accepted and _Default_ Values | Description | |---------------|-------------------------------|------------------------------------------------------| -| alwaysUse | boolean, _false_ | Enforce that the sandbox be used on every acion. | +| alwaysUseSandbox | boolean, _false_ | Enforce that the sandbox be used on every acion. | +| alwaysUseCgroups | boolean, _true_ | Enforce that actions run under cgroups. | | alwaysUseTmpFs | boolean, _false_ | Enforce that the sandbox uses tmpfs on every acion. | | selectForBlockNetwork | boolean, _false_ | `block-network` enables sandbox action execution. | | selectForTmpFs | boolean, _false_ | `tmpfs` enables sandbox action execution. | @@ -303,12 +304,15 @@ Example: ```yaml worker: sandboxSettings: - alwaysUse: true + alwaysUseSandbox: true + alwaysUseCgroups: true alwaysUseTmpFs: true selectForBlockNetwork: false selectForTmpFs: false ``` +Note: In order for these settings to take effect, you must also configure `limitGlobalExecution: true`. + ### Dequeue Match | Configuration | Accepted and _Default_ Values | Description | diff --git a/examples/config.yml b/examples/config.yml index b0626f467..d69688b5b 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -129,7 +129,8 @@ worker: errorOperationOutputSizeExceeded: false gracefulShutdownSeconds: 0 sandboxSettings: - alwaysUse: false + alwaysUseSandbox: false + alwaysUseCgroups: false alwaysUseTmpFs: false selectForBlockNetwork: false selectForTmpFs: false diff --git a/src/main/java/build/buildfarm/common/config/SandboxSettings.java b/src/main/java/build/buildfarm/common/config/SandboxSettings.java index fc56aa889..f43298901 100644 --- a/src/main/java/build/buildfarm/common/config/SandboxSettings.java +++ b/src/main/java/build/buildfarm/common/config/SandboxSettings.java @@ -26,11 +26,11 @@ @Data public class SandboxSettings { /** - * @field alwaysUse + * @field alwaysUseSandbox * @brief Whether or not to always use the sandbox when running actions. * @details It may be preferred to enforce sandbox usage than rely on client selection. */ - public boolean alwaysUse = false; + public boolean alwaysUseSandbox = false; /** * @field alwaysUseCgroups diff --git a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java index 78021f1ed..5523f1aa7 100644 --- a/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java +++ b/src/main/java/build/buildfarm/worker/resources/ResourceDecider.java @@ -200,10 +200,9 @@ private static void adjustLimits( } private static void decideSandboxUsage(ResourceLimits limits, SandboxSettings sandbox) { - // Decide which sandbox limitations are enabled by default acording to the deployment's // configuration. - if (sandbox.isAlwaysUse()) { + if (sandbox.isAlwaysUseSandbox()) { limits.useLinuxSandbox = true; limits.description.add("enabled sandbox by default"); } diff --git a/src/test/java/build/buildfarm/worker/resources/ResourceDeciderTest.java b/src/test/java/build/buildfarm/worker/resources/ResourceDeciderTest.java index 8fa711ab5..9c531c7e7 100644 --- a/src/test/java/build/buildfarm/worker/resources/ResourceDeciderTest.java +++ b/src/test/java/build/buildfarm/worker/resources/ResourceDeciderTest.java @@ -726,7 +726,7 @@ public void decideResourceLimitationsAlwaysUseSandbox() throws Exception { // ARRANGE Command command = Command.newBuilder().build(); SandboxSettings sandboxSettings = new SandboxSettings(); - sandboxSettings.alwaysUse = true; + sandboxSettings.alwaysUseSandbox = true; // ACT ResourceLimits limits = From 72b40ae5c3bf757e9129bbea272fc9a2f5b94bab Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Sun, 23 Apr 2023 13:41:39 -0400 Subject: [PATCH 42/68] updates --- .../worker/shard/ShardWorkerContext.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index 590d492de..ad46a7cae 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -829,7 +829,9 @@ boolean shouldLimitCoreUsage() { @Override public void createExecutionLimits() { if (shouldLimitCoreUsage()) { - createOperationExecutionLimits(); + if (configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { + createOperationExecutionLimits(); + } } } @@ -860,11 +862,13 @@ void createOperationExecutionLimits() { @Override public void destroyExecutionLimits() { - try { - operationsGroup.getCpu().close(); - executionsGroup.getCpu().close(); - } catch (IOException e) { - throw new RuntimeException(e); + if (configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { + try { + operationsGroup.getCpu().close(); + executionsGroup.getCpu().close(); + } catch (IOException e) { + throw new RuntimeException(e); + } } } From 14c759b4be181bb3bffa8adc18c0deb87a70c3fb Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Thu, 9 Nov 2023 05:15:56 +0800 Subject: [PATCH 43/68] Update ShardWorkerContext.java --- .../java/build/buildfarm/worker/shard/ShardWorkerContext.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index ad46a7cae..a8058f6f5 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -828,10 +828,8 @@ boolean shouldLimitCoreUsage() { @Override public void createExecutionLimits() { - if (shouldLimitCoreUsage()) { - if (configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { + if (shouldLimitCoreUsage() && configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { createOperationExecutionLimits(); - } } } From ca9bb92b9f72f463d7d653298e2ca2619c451e2e Mon Sep 17 00:00:00 2001 From: Trevor Hickey Date: Thu, 9 Nov 2023 05:17:08 +0800 Subject: [PATCH 44/68] Update ShardWorkerContext.java --- .../java/build/buildfarm/worker/shard/ShardWorkerContext.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index a8058f6f5..11f5e49b3 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -828,8 +828,8 @@ boolean shouldLimitCoreUsage() { @Override public void createExecutionLimits() { - if (shouldLimitCoreUsage() && configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { - createOperationExecutionLimits(); + if (shouldLimitCoreUsage() && configs.getWorker().getSandboxSettings().isAlwaysUseCgroups()) { + createOperationExecutionLimits(); } } From 35883a4d064c33f2c507e51c11661ca761543e44 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 9 Nov 2023 13:25:39 -0500 Subject: [PATCH 45/68] Release resources when not keeping an operation (#1535) --- .../worker/DequeueMatchEvaluator.java | 18 +++++---- .../buildfarm/worker/DequeueResults.java | 38 ++++++++++++++++++ .../worker/shard/ShardWorkerContext.java | 18 ++++++--- .../worker/DequeueMatchEvaluatorTest.java | 39 ++++++++++++------- 4 files changed, 88 insertions(+), 25 deletions(-) create mode 100644 src/main/java/build/buildfarm/worker/DequeueResults.java diff --git a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java index 59ccd00e3..997fc6862 100644 --- a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java +++ b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java @@ -55,7 +55,7 @@ public class DequeueMatchEvaluator { */ @SuppressWarnings("NullableProblems") @NotNull - public static boolean shouldKeepOperation( + public static DequeueResults shouldKeepOperation( SetMultimap workerProvisions, String name, LocalResourceSet resourceSet, @@ -77,28 +77,32 @@ public static boolean shouldKeepOperation( */ @SuppressWarnings("NullableProblems") @NotNull - private static boolean shouldKeepViaPlatform( + private static DequeueResults shouldKeepViaPlatform( SetMultimap workerProvisions, String name, LocalResourceSet resourceSet, Platform platform) { // attempt to execute everything the worker gets off the queue, // provided there is enough resources to do so. - // this is a recommended configuration. + DequeueResults results = new DequeueResults(); + if (!LocalResourceSetUtils.claimResources(platform, resourceSet)) { - return false; + return results; } + results.resourcesClaimed = true; // The action might be requesting to run on a particular action if (!keepForThisWorker(platform, name)) { - return false; + return results; } if (configs.getWorker().getDequeueMatchSettings().isAcceptEverything()) { - return true; + results.keep = true; + return results; } - return satisfiesProperties(workerProvisions, platform); + results.keep = satisfiesProperties(workerProvisions, platform); + return results; } private static boolean keepForThisWorker(Platform platform, String name) { diff --git a/src/main/java/build/buildfarm/worker/DequeueResults.java b/src/main/java/build/buildfarm/worker/DequeueResults.java new file mode 100644 index 000000000..f449907db --- /dev/null +++ b/src/main/java/build/buildfarm/worker/DequeueResults.java @@ -0,0 +1,38 @@ +// Copyright 2021 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.worker; + +/** + * @class DequeueResults + * @brief The results of evaluating a dequeued operation. + * @details Contains information about whether the operation should be kept and if resources were + * claimed. + */ +public class DequeueResults { + /** + * @field keep + * @brief Whether the operation should be kept. + * @details Operations not kept should be put back on the queue. + */ + public boolean keep = false; + + /** + * @field resourcesClaimed + * @brief Whether resources have been claimed while deciding to keep operation. + * @details If resources are claimed but the operation is not kept, the resources should be + * released. + */ + public boolean resourcesClaimed = false; +} diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index 11f5e49b3..af380750c 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -55,6 +55,7 @@ import build.buildfarm.v1test.QueueEntry; import build.buildfarm.v1test.QueuedOperation; import build.buildfarm.worker.DequeueMatchEvaluator; +import build.buildfarm.worker.DequeueResults; import build.buildfarm.worker.ExecutionPolicies; import build.buildfarm.worker.RetryingMatchListener; import build.buildfarm.worker.WorkerContext; @@ -283,10 +284,14 @@ public QueuedOperation getQueuedOperation(QueueEntry queueEntry) @SuppressWarnings("ConstantConditions") private void matchInterruptible(MatchListener listener) throws IOException, InterruptedException { QueueEntry queueEntry = takeEntryOffOperationQueue(listener); - decideWhetherToKeepOperation(queueEntry, listener); + if (queueEntry == null) { + listener.onEntry(null); + } else { + decideWhetherToKeepOperation(queueEntry, listener); + } } - private QueueEntry takeEntryOffOperationQueue(MatchListener listener) + private @Nullable QueueEntry takeEntryOffOperationQueue(MatchListener listener) throws IOException, InterruptedException { listener.onWaitStart(); QueueEntry queueEntry = null; @@ -314,11 +319,14 @@ private QueueEntry takeEntryOffOperationQueue(MatchListener listener) private void decideWhetherToKeepOperation(QueueEntry queueEntry, MatchListener listener) throws IOException, InterruptedException { - if (queueEntry == null - || DequeueMatchEvaluator.shouldKeepOperation( - matchProvisions, name, resourceSet, queueEntry)) { + DequeueResults results = + DequeueMatchEvaluator.shouldKeepOperation(matchProvisions, name, resourceSet, queueEntry); + if (results.keep) { listener.onEntry(queueEntry); } else { + if (results.resourcesClaimed) { + returnLocalResources(queueEntry); + } backplane.rejectOperation(queueEntry); } if (Thread.interrupted()) { diff --git a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java index 290470b84..112d7a63e 100644 --- a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java +++ b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java @@ -60,7 +60,8 @@ public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT assertThat(shouldKeep).isTrue(); @@ -89,7 +90,8 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker accepts because it has more cores than the min-cores requested @@ -120,7 +122,8 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker rejects because it has less cores than the min-cores requested @@ -150,7 +153,8 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker accepts because it has the same cores as the min-cores requested @@ -180,7 +184,8 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT assertThat(shouldKeep).isFalse(); @@ -191,7 +196,8 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E // ACT shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT assertThat(shouldKeep).isTrue(); @@ -202,7 +208,8 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E // ACT shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT assertThat(shouldKeep).isTrue(); @@ -234,7 +241,8 @@ public void shouldKeepOperationClaimsResource() throws Exception { // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker accepts because the resource is available. @@ -244,7 +252,8 @@ public void shouldKeepOperationClaimsResource() throws Exception { // ACT shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker rejects because there are no resources left. @@ -282,7 +291,8 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker accepts because the resource is available. @@ -293,7 +303,8 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ACT shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker accepts because the resource is available. @@ -304,7 +315,8 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ACT shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker rejects because there are no resources left. @@ -348,7 +360,8 @@ public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception // ACT boolean shouldKeep = DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry); + workerProvisions, "worker-name", resourceSet, entry) + .keep; // ASSERT // the worker rejects because there are no resources left. From 9d80f4ee8726dbdc7578ac7307038f503aace901 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 9 Nov 2023 15:35:41 -0500 Subject: [PATCH 46/68] Update queues.md Refer to new camelized DMS fields. Expand predefined dynamic execution property name matches. --- _site/docs/architecture/queues.md | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/_site/docs/architecture/queues.md b/_site/docs/architecture/queues.md index 872b68973..2dc706bf4 100644 --- a/_site/docs/architecture/queues.md +++ b/_site/docs/architecture/queues.md @@ -25,33 +25,38 @@ If your configuration file does not specify any provisioned queues, buildfarm wi This will ensure the expected behavior for the paradigm in which all work is put on the same queue. ### Matching Algorithm -The matching algorithm is performed by the operation queue when the caller is requesting to push or pop elements. +The matching algorithm is performed by the operation queue when the server or worker is requesting to push or pop elements, respectively. The matching algorithm is designed to find the appropriate queue to perform these actions on. On the scheduler side, the action's platform properties are used for matching. On the worker side, the `dequeue_match_settings` are used. ![Operation Queue Matching]({{site.url}}{{site.baseurl}}/assets/images/Operation-Queue-Matching1.png) -This is how the matching algorithm works: +The matching algorithm works as follows: Each provision queue is checked in the order that it is configured. The first provision queue that is deemed eligible is chosen and used. When deciding if an action is eligible for the provision queue, each platform property is checked individually. By default, there must be a perfect match on each key/value. Wildcards ("*") can be used to avoid the need of a perfect match. Additionally, if the action contains any platform properties is not mentioned by the provision queue, it will be deemed ineligible. -setting `allow_unmatched: true` can be used to allow a superset of action properties as long as a subset matches the provision queue. +setting `allowUnmatched: true` can be used to allow a superset of action properties as long as a subset matches the provision queue. If no provision queues can be matched, the operation queue will provide an analysis on why none of the queues were eligible. -When taking elements off of the operation queue, the matching algorithm behaves a similar way. -The worker's `DequeueMatchSettings` also have an `allow_unmatched` property. +When taking elements off of the operation queue, the worker's matching algorithm behaves in a similar manner: +The worker's `DequeueMatchSettings` also have an `allowUnmatched` property. Workers also have the ability to reject an operation after matching with a provision queue and dequeuing a value. -To avoid any of these rejections by the worker, you can use `accept_everything: true`. +To avoid any of these rejections by the worker, you can use `acceptEverything: true`. When configuring your worker, consider the following decisions: -First, if the accept_everything setting is true, the job is accepted. +First, if the `allowEverything` setting is `true`, the job is accepted. Otherwise, if any execution property for the queue has a wildcard key, the job is accepted. -Otherwise, if the allow_unmatched setting is true, each key present in the queue's properties must be a wildcard or exist in the execution request's properties with an equal value. +Otherwise, if the `allowUnmatched` setting is `true`, each key present in the queue's properties must be a wildcard or exist in the execution request's properties with an equal value. Otherwise, the execution request's properties must have exactly the same set of keys as the queue's execution properties, and the request's value for each property must equal the queue's if the queue's value for this property is not a wildcard. +There are special predefined execution property names which resolve to dynamic configuration for the worker to match against: +`Worker`: The worker's `publicName` with no wildcard resolution +`min-cores`: Less than or equal to the `executeStageWidth` +`process-wrapper`: The set of named `process-wrappers` present in configuration + ### Server Example In this example the scheduler declares a GPU queue and CPU queue. All queues must be declared for the server deployment: From aac33b60ee36e71a9f4facafa5f704edbd02fc25 Mon Sep 17 00:00:00 2001 From: Matas Rastenis Date: Thu, 9 Nov 2023 16:52:21 -0800 Subject: [PATCH 47/68] Implement custom label header support for Grpc metrics interceptor (#1530) Add an option to provide a list of custom label headers to add to metrics. --- _site/docs/configuration/configuration.md | 2 ++ examples/config.yml | 1 + .../java/build/buildfarm/common/config/GrpcMetrics.java | 7 +++++++ 3 files changed, 10 insertions(+) diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index 83d4dcc8d..d51a1f229 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -94,6 +94,7 @@ server: |--------------------------|-------------------------------|--------------------------------------------------------| | enabled | boolean, _false_ | Publish basic GRPC metrics to a Prometheus endpoint | | provideLatencyHistograms | boolean, _false_ | Publish detailed, more expensive to calculate, metrics | +| labelsToReport | List of Strings, _[]_ | Include custom metrics labels in Prometheus metrics | Example: @@ -102,6 +103,7 @@ server: grpcMetrics: enabled: false provideLatencyHistograms: false + labelsToReport: [] ``` ### Server Caches diff --git a/examples/config.yml b/examples/config.yml index d69688b5b..825121ea4 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -13,6 +13,7 @@ server: enabled: true provideLatencyHistograms: true latencyBuckets: [0.001, 0.01, 0.1, 1, 5, 10, 20, 40, 60, +Infinity] + labelsToReport: [] maxInboundMessageSizeBytes: 0 maxInboundMetadataSize: 0 casWriteTimeout: 3600 diff --git a/src/main/java/build/buildfarm/common/config/GrpcMetrics.java b/src/main/java/build/buildfarm/common/config/GrpcMetrics.java index 8855ae5fd..a6e865350 100644 --- a/src/main/java/build/buildfarm/common/config/GrpcMetrics.java +++ b/src/main/java/build/buildfarm/common/config/GrpcMetrics.java @@ -1,6 +1,7 @@ package build.buildfarm.common.config; import io.grpc.ServerBuilder; +import java.util.List; import lombok.Data; import me.dinowernli.grpc.prometheus.Configuration; import me.dinowernli.grpc.prometheus.MonitoringServerInterceptor; @@ -10,6 +11,7 @@ public class GrpcMetrics { private boolean enabled = false; private boolean provideLatencyHistograms = false; private double[] latencyBuckets; + private List labelsToReport; public static void handleGrpcMetricIntercepts( ServerBuilder serverBuilder, GrpcMetrics grpcMetrics) { @@ -30,6 +32,11 @@ public static void handleGrpcMetricIntercepts( grpcConfig = grpcConfig.withLatencyBuckets(grpcMetrics.getLatencyBuckets()); } + // report custom metric labels + if (grpcMetrics.getLabelsToReport() != null) { + grpcConfig = grpcConfig.withLabelHeaders(grpcMetrics.getLabelsToReport()); + } + // Apply config to create an interceptor and apply it to the GRPC server. MonitoringServerInterceptor monitoringInterceptor = MonitoringServerInterceptor.create(grpcConfig); From f9882f7454c39c17fb957c2eaa7538695a41b255 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Sat, 11 Nov 2023 00:36:03 -0500 Subject: [PATCH 48/68] Specify direct guava dependency usage (#1538) Testing with bazel HEAD using jdk21 compilation has revealed new direct dependencies on guava. --- persistentworkers/src/test/java/persistent/bazel/BUILD | 1 + src/test/java/build/buildfarm/common/grpc/BUILD | 1 + src/test/java/build/buildfarm/common/io/BUILD | 1 + src/test/java/build/buildfarm/common/redis/BUILD | 1 + src/test/java/build/buildfarm/common/services/BUILD | 1 + src/test/java/build/buildfarm/server/services/BUILD | 1 + 6 files changed, 6 insertions(+) diff --git a/persistentworkers/src/test/java/persistent/bazel/BUILD b/persistentworkers/src/test/java/persistent/bazel/BUILD index 0cf829a11..b85db7800 100644 --- a/persistentworkers/src/test/java/persistent/bazel/BUILD +++ b/persistentworkers/src/test/java/persistent/bazel/BUILD @@ -3,6 +3,7 @@ COMMON_DEPS = [ "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", "//persistentworkers/src/test/java/persistent/testutil:testutil", "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", + "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", "@maven//:com_google_truth_truth", "@maven//:commons_io_commons_io", diff --git a/src/test/java/build/buildfarm/common/grpc/BUILD b/src/test/java/build/buildfarm/common/grpc/BUILD index d470ee243..35be7f36d 100644 --- a/src/test/java/build/buildfarm/common/grpc/BUILD +++ b/src/test/java/build/buildfarm/common/grpc/BUILD @@ -23,6 +23,7 @@ java_test( "//src/test/java/build/buildfarm:test_runner", "@googleapis//:google_bytestream_bytestream_java_grpc", "@googleapis//:google_bytestream_bytestream_java_proto", + "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", "@maven//:com_google_truth_truth", "@maven//:io_grpc_grpc_api", diff --git a/src/test/java/build/buildfarm/common/io/BUILD b/src/test/java/build/buildfarm/common/io/BUILD index 551b96ea5..1f5d1ca17 100644 --- a/src/test/java/build/buildfarm/common/io/BUILD +++ b/src/test/java/build/buildfarm/common/io/BUILD @@ -8,6 +8,7 @@ java_test( "//src/test/java/build/buildfarm:test_runner", "@googleapis//:google_bytestream_bytestream_java_grpc", "@googleapis//:google_bytestream_bytestream_java_proto", + "@maven//:com_google_guava_guava", "@maven//:com_google_jimfs_jimfs", "@maven//:com_google_protobuf_protobuf_java", "@maven//:com_google_truth_truth", diff --git a/src/test/java/build/buildfarm/common/redis/BUILD b/src/test/java/build/buildfarm/common/redis/BUILD index 7b3978fb5..f388e339e 100644 --- a/src/test/java/build/buildfarm/common/redis/BUILD +++ b/src/test/java/build/buildfarm/common/redis/BUILD @@ -6,6 +6,7 @@ COMMON_DEPS = [ "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "//src/test/java/build/buildfarm:test_runner", "//third_party/jedis", + "@maven//:com_google_guava_guava", "@maven//:com_google_truth_truth", "@maven//:io_grpc_grpc_api", "@maven//:org_mockito_mockito_core", diff --git a/src/test/java/build/buildfarm/common/services/BUILD b/src/test/java/build/buildfarm/common/services/BUILD index 2ba54ab58..6fe1f35c5 100644 --- a/src/test/java/build/buildfarm/common/services/BUILD +++ b/src/test/java/build/buildfarm/common/services/BUILD @@ -16,6 +16,7 @@ java_test( "//src/test/java/build/buildfarm:test_runner", "@googleapis//:google_bytestream_bytestream_java_grpc", "@googleapis//:google_bytestream_bytestream_java_proto", + "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", "@maven//:com_google_truth_truth", "@maven//:io_grpc_grpc_api", diff --git a/src/test/java/build/buildfarm/server/services/BUILD b/src/test/java/build/buildfarm/server/services/BUILD index 09916ed7f..a83385da0 100644 --- a/src/test/java/build/buildfarm/server/services/BUILD +++ b/src/test/java/build/buildfarm/server/services/BUILD @@ -11,6 +11,7 @@ java_test( "//src/main/java/build/buildfarm/server/services", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "//src/test/java/build/buildfarm:test_runner", + "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", "@maven//:com_google_truth_truth", "@maven//:io_grpc_grpc_stub", From 69e0248b64256de2f519866cc6dd33769a18ff24 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Sat, 11 Nov 2023 00:38:29 -0500 Subject: [PATCH 49/68] Update lombok dependency for jdk21 (#1540) Annotations under lombok were fixed for jdk21 in 1.18.28, update to current. --- defs.bzl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/defs.bzl b/defs.bzl index a40dccaef..ea145d1b9 100644 --- a/defs.bzl +++ b/defs.bzl @@ -112,7 +112,7 @@ def buildfarm_init(name = "buildfarm"): "org.xerial:sqlite-jdbc:3.34.0", "org.jetbrains:annotations:16.0.2", "org.yaml:snakeyaml:2.0", - "org.projectlombok:lombok:1.18.24", + "org.projectlombok:lombok:1.18.30", ], generate_compat_repositories = True, override_targets = IO_GRPC_GRPC_JAVA_OVERRIDE_TARGETS, From 339aa131a971c1e541ede590ea8096cfbde97824 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Sat, 11 Nov 2023 00:47:00 -0500 Subject: [PATCH 50/68] Reorganize DequeueMatchEvaluator (#1537) Remove acceptEverything DequeueMatchSetting Place worker name in workerProvisions Only enable allowUnmatched effects on key mismatch Only acquire resources after asserting compatibility Update documentation to match changes --- _site/docs/architecture/queues.md | 20 ++-- _site/docs/configuration/configuration.md | 2 - examples/config.yml | 1 - .../common/config/DequeueMatchSettings.java | 7 +- .../worker/DequeueMatchEvaluator.java | 54 ++-------- .../worker/shard/ShardWorkerContext.java | 32 ++---- .../worker/DequeueMatchEvaluatorTest.java | 102 ++++++++---------- 7 files changed, 79 insertions(+), 139 deletions(-) diff --git a/_site/docs/architecture/queues.md b/_site/docs/architecture/queues.md index 2dc706bf4..4ee44764b 100644 --- a/_site/docs/architecture/queues.md +++ b/_site/docs/architecture/queues.md @@ -41,19 +41,17 @@ Additionally, if the action contains any platform properties is not mentioned by setting `allowUnmatched: true` can be used to allow a superset of action properties as long as a subset matches the provision queue. If no provision queues can be matched, the operation queue will provide an analysis on why none of the queues were eligible. -When taking elements off of the operation queue, the worker's matching algorithm behaves in a similar manner: -The worker's `DequeueMatchSettings` also have an `allowUnmatched` property. -Workers also have the ability to reject an operation after matching with a provision queue and dequeuing a value. -To avoid any of these rejections by the worker, you can use `acceptEverything: true`. - -When configuring your worker, consider the following decisions: -First, if the `allowEverything` setting is `true`, the job is accepted. -Otherwise, if any execution property for the queue has a wildcard key, the job is accepted. -Otherwise, if the `allowUnmatched` setting is `true`, each key present in the queue's properties must be a wildcard or exist in the execution request's properties with an equal value. -Otherwise, the execution request's properties must have exactly the same set of keys as the queue's execution properties, and the request's value for each property must equal the queue's if the queue's value for this property is not a wildcard. +A worker will dequeue operations from matching queues and determine whether to keep and execute it according to the following procedure: +For each property key-value in the operation's platform, an operation is REJECTED if: + The key is `min-cores` and the integer value is greater than the number of cores on the worker. + Or The key is `min-mem` and the integer value is greater than the number of bytes of RAM on the worker. + Or if the key exists in the `DequeueMatchSettings` platform with neither the value nor a `*` in the corresponding DMS platform key's values, + Or if the `allowUnmatched` setting is `false`. +For each resource requested in the operation's platform with the resource: prefix, the action is rejected if: + The resource amount cannot currently be satisfied with the associated resource capacity count There are special predefined execution property names which resolve to dynamic configuration for the worker to match against: -`Worker`: The worker's `publicName` with no wildcard resolution +`Worker`: The worker's `publicName` `min-cores`: Less than or equal to the `executeStageWidth` `process-wrapper`: The set of named `process-wrappers` present in configuration diff --git a/_site/docs/configuration/configuration.md b/_site/docs/configuration/configuration.md index d51a1f229..ebbb8ca0e 100644 --- a/_site/docs/configuration/configuration.md +++ b/_site/docs/configuration/configuration.md @@ -319,7 +319,6 @@ Note: In order for these settings to take effect, you must also configure `limit | Configuration | Accepted and _Default_ Values | Description | |------------------|-------------------------------|-------------| -| acceptEverything | boolean, _true_ | | | allowUnmatched | boolean, _false_ | | Example: @@ -327,7 +326,6 @@ Example: ```yaml worker: dequeueMatchSettings: - acceptEverything: true allowUnmatched: false ``` diff --git a/examples/config.yml b/examples/config.yml index 825121ea4..c02ff4cfc 100644 --- a/examples/config.yml +++ b/examples/config.yml @@ -103,7 +103,6 @@ worker: inlineContentLimit: 1048567 # 1024 * 1024 operationPollPeriod: 1 dequeueMatchSettings: - acceptEverything: true allowUnmatched: false storages: - type: FILESYSTEM diff --git a/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java b/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java index 20ccbf85b..57aad7783 100644 --- a/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java +++ b/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java @@ -3,11 +3,16 @@ import build.bazel.remote.execution.v2.Platform; import java.util.ArrayList; import java.util.List; +import lombok.AccessLevel; import lombok.Data; +import lombok.Getter; @Data public class DequeueMatchSettings { - private boolean acceptEverything = true; + + @Getter(AccessLevel.NONE) + private boolean acceptEverything; // deprecated + private boolean allowUnmatched = false; private List properties = new ArrayList(); diff --git a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java index 997fc6862..3f5815fd1 100644 --- a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java +++ b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java @@ -55,12 +55,11 @@ public class DequeueMatchEvaluator { */ @SuppressWarnings("NullableProblems") @NotNull - public static DequeueResults shouldKeepOperation( + public static boolean shouldKeepOperation( SetMultimap workerProvisions, - String name, LocalResourceSet resourceSet, QueueEntry queueEntry) { - return shouldKeepViaPlatform(workerProvisions, name, resourceSet, queueEntry.getPlatform()); + return shouldKeepViaPlatform(workerProvisions, resourceSet, queueEntry.getPlatform()); } /** @@ -77,43 +76,12 @@ public static DequeueResults shouldKeepOperation( */ @SuppressWarnings("NullableProblems") @NotNull - private static DequeueResults shouldKeepViaPlatform( + private static boolean shouldKeepViaPlatform( SetMultimap workerProvisions, - String name, LocalResourceSet resourceSet, Platform platform) { - // attempt to execute everything the worker gets off the queue, - // provided there is enough resources to do so. - DequeueResults results = new DequeueResults(); - - if (!LocalResourceSetUtils.claimResources(platform, resourceSet)) { - return results; - } - results.resourcesClaimed = true; - - // The action might be requesting to run on a particular action - if (!keepForThisWorker(platform, name)) { - return results; - } - - if (configs.getWorker().getDequeueMatchSettings().isAcceptEverything()) { - results.keep = true; - return results; - } - - results.keep = satisfiesProperties(workerProvisions, platform); - return results; - } - - private static boolean keepForThisWorker(Platform platform, String name) { - for (Platform.Property property : platform.getPropertiesList()) { - if (property.getName().equals(ExecutionProperties.WORKER) - && !property.getValue().equals(name)) { - // requested worker does not match this worker, reject - return false; - } - } - return true; + return satisfiesProperties(workerProvisions, platform) + && LocalResourceSetUtils.claimResources(platform, resourceSet); } /** @@ -183,13 +151,13 @@ private static boolean satisfiesProperty( return possibleMemories >= memBytesRequested; } - // accept other properties not specified on the worker - if (configs.getWorker().getDequeueMatchSettings().isAllowUnmatched()) { - return true; + // ensure exact matches + if (workerProvisions.containsKey(property.getName())) { + return workerProvisions.containsEntry(property.getName(), property.getValue()) + || workerProvisions.containsEntry(property.getName(), "*"); } - // ensure exact matches - return workerProvisions.containsEntry(property.getName(), property.getValue()) - || workerProvisions.containsEntry(property.getName(), "*"); + // accept other properties not specified on the worker + return configs.getWorker().getDequeueMatchSettings().isAllowUnmatched(); } } diff --git a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java index af380750c..ac020f504 100644 --- a/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java +++ b/src/main/java/build/buildfarm/worker/shard/ShardWorkerContext.java @@ -18,6 +18,7 @@ import static build.buildfarm.common.Actions.checkPreconditionFailure; import static build.buildfarm.common.Errors.VIOLATION_TYPE_INVALID; import static build.buildfarm.common.Errors.VIOLATION_TYPE_MISSING; +import static build.buildfarm.worker.DequeueMatchEvaluator.shouldKeepOperation; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.DAYS; @@ -38,6 +39,7 @@ import build.buildfarm.common.DigestUtil; import build.buildfarm.common.DigestUtil.ActionKey; import build.buildfarm.common.EntryLimitException; +import build.buildfarm.common.ExecutionProperties; import build.buildfarm.common.InputStreamFactory; import build.buildfarm.common.LinuxSandboxOptions; import build.buildfarm.common.Poller; @@ -54,8 +56,6 @@ import build.buildfarm.v1test.CASInsertionPolicy; import build.buildfarm.v1test.QueueEntry; import build.buildfarm.v1test.QueuedOperation; -import build.buildfarm.worker.DequeueMatchEvaluator; -import build.buildfarm.worker.DequeueResults; import build.buildfarm.worker.ExecutionPolicies; import build.buildfarm.worker.RetryingMatchListener; import build.buildfarm.worker.WorkerContext; @@ -137,7 +137,7 @@ class ShardWorkerContext implements WorkerContext { private final boolean errorOperationOutputSizeExceeded; static SetMultimap getMatchProvisions( - Iterable policies, int executeStageWidth) { + Iterable policies, String name, int executeStageWidth) { ImmutableSetMultimap.Builder provisions = ImmutableSetMultimap.builder(); Platform matchPlatform = ExecutionPolicies.getMatchPlatform( @@ -146,6 +146,7 @@ static SetMultimap getMatchProvisions( provisions.put(property.getName(), property.getValue()); } provisions.put(PROVISION_CORES_NAME, String.format("%d", executeStageWidth)); + provisions.put(ExecutionProperties.WORKER, name); return provisions.build(); } @@ -172,7 +173,7 @@ static SetMultimap getMatchProvisions( LocalResourceSet resourceSet, CasWriter writer) { this.name = name; - this.matchProvisions = getMatchProvisions(policies, executeStageWidth); + this.matchProvisions = getMatchProvisions(policies, name, executeStageWidth); this.operationPollPeriod = operationPollPeriod; this.operationPoller = operationPoller; this.inputFetchStageWidth = inputFetchStageWidth; @@ -284,10 +285,10 @@ public QueuedOperation getQueuedOperation(QueueEntry queueEntry) @SuppressWarnings("ConstantConditions") private void matchInterruptible(MatchListener listener) throws IOException, InterruptedException { QueueEntry queueEntry = takeEntryOffOperationQueue(listener); - if (queueEntry == null) { - listener.onEntry(null); + if (queueEntry == null || shouldKeepOperation(matchProvisions, resourceSet, queueEntry)) { + listener.onEntry(queueEntry); } else { - decideWhetherToKeepOperation(queueEntry, listener); + backplane.rejectOperation(queueEntry); } } @@ -317,23 +318,6 @@ private void matchInterruptible(MatchListener listener) throws IOException, Inte return queueEntry; } - private void decideWhetherToKeepOperation(QueueEntry queueEntry, MatchListener listener) - throws IOException, InterruptedException { - DequeueResults results = - DequeueMatchEvaluator.shouldKeepOperation(matchProvisions, name, resourceSet, queueEntry); - if (results.keep) { - listener.onEntry(queueEntry); - } else { - if (results.resourcesClaimed) { - returnLocalResources(queueEntry); - } - backplane.rejectOperation(queueEntry); - } - if (Thread.interrupted()) { - throw new InterruptedException(); - } - } - @Override public void returnLocalResources(QueueEntry queueEntry) { LocalResourceSetUtils.releaseClaims(queueEntry.getPlatform(), resourceSet); diff --git a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java index 112d7a63e..4d484b8c4 100644 --- a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java +++ b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java @@ -59,9 +59,7 @@ public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -89,9 +87,7 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because it has more cores than the min-cores requested @@ -106,7 +102,6 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { @Test public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(false); SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); workerProvisions.put("cores", "10"); @@ -121,9 +116,7 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because it has less cores than the min-cores requested @@ -152,9 +145,7 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because it has the same cores as the min-cores requested @@ -168,7 +159,6 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti @Test public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws Exception { // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(false); configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(false); SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); @@ -183,33 +173,16 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isFalse(); - // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); - - // ACT - shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; - - // ASSERT - assertThat(shouldKeep).isTrue(); - // ARRANGE configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); // ACT - shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -221,7 +194,6 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E @Test public void shouldKeepOperationClaimsResource() throws Exception { // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); @@ -240,9 +212,7 @@ public void shouldKeepOperationClaimsResource() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -250,10 +220,7 @@ public void shouldKeepOperationClaimsResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); // ACT - shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -261,13 +228,45 @@ public void shouldKeepOperationClaimsResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); } + // Function under test: shouldKeepOperation + // Reason for testing: the local resource should be claimed + // Failure explanation: semaphore claim did not work as expected. + @Test + public void rejectOperationIgnoresResource() throws Exception { + // ARRANGE + configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(false); + SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); + resourceSet.resources.put("FOO", new Semaphore(1)); + + QueueEntry entry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties( + Platform.Property.newBuilder().setName("resource:FOO").setValue("1")) + .addProperties(Platform.Property.newBuilder().setName("os").setValue("randos"))) + .build(); + + // PRE-ASSERT + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); + + // ACT + boolean shouldKeep = + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + + // ASSERT + // the worker rejects because the os is not satisfied + assertThat(shouldKeep).isFalse(); + assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); + } + // Function under test: shouldKeepOperation // Reason for testing: the local resources should be claimed // Failure explanation: semaphore claim did not work as expected. @Test public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); @@ -290,9 +289,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -301,10 +298,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(2); // ACT - shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -313,10 +307,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(0); // ACT - shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -332,7 +323,6 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { @Test public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception { // ARRANGE - configs.getWorker().getDequeueMatchSettings().setAcceptEverything(true); configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); @@ -359,9 +349,7 @@ public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception // ACT boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation( - workerProvisions, "worker-name", resourceSet, entry) - .keep; + DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. From 025305ad9dd06c4f04e702dadb150e314d15c826 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Sat, 11 Nov 2023 08:54:51 -0500 Subject: [PATCH 51/68] Upgrade com_google_protobuf for jvm compatibility (#1539) Correct deprecated AccessController usage warning Requires a newer bazel than 6.4.0 for macos to choose unix toolchain with C++ std=c++14 specification for protobuf->absl dependency. --- .bazelci/presubmit.yml | 3 +++ deps.bzl | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/.bazelci/presubmit.yml b/.bazelci/presubmit.yml index 728e68d02..e08f2f07b 100644 --- a/.bazelci/presubmit.yml +++ b/.bazelci/presubmit.yml @@ -55,7 +55,10 @@ tasks: - "..." macos: name: "Unit Tests" + environment: + USE_BAZEL_VERSION: 17be878292730359c9c90efdceabed26126df7ae build_flags: + - "--cxxopt=-std=c++14" - "--build_tag_filters=-container" build_targets: - "..." diff --git a/deps.bzl b/deps.bzl index 55b12ab13..51dfbb953 100644 --- a/deps.bzl +++ b/deps.bzl @@ -36,9 +36,9 @@ def archive_dependencies(third_party): # Needed for "well-known protos" and @com_google_protobuf//:protoc. { "name": "com_google_protobuf", - "sha256": "25f1292d4ea6666f460a2a30038eef121e6c3937ae0f61d610611dfb14b0bd32", - "strip_prefix": "protobuf-3.19.1", - "urls": ["https://github.com/protocolbuffers/protobuf/archive/v3.19.1.zip"], + "sha256": "79082dc68d8bab2283568ce0be3982b73e19ddd647c2411d1977ca5282d2d6b3", + "strip_prefix": "protobuf-25.0", + "urls": ["https://github.com/protocolbuffers/protobuf/archive/v25.0.zip"], }, { "name": "com_github_bazelbuild_buildtools", From f7209098f5d1b4fc671d3008af22aafaa8689902 Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Sat, 11 Nov 2023 09:12:37 -0500 Subject: [PATCH 52/68] Create buildfarm-worker-base-build-and-deploy.yml (#1534) Create a github workflow to build base buildfarm worker image. --- ...buildfarm-worker-base-build-and-deploy.yml | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 .github/workflows/buildfarm-worker-base-build-and-deploy.yml diff --git a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml new file mode 100644 index 000000000..7550639dd --- /dev/null +++ b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml @@ -0,0 +1,28 @@ +name: Build and push base Buildfarm worker image + +on: + push: + branches: + - main + paths: + - ci/base-worker-image/jammy/Dockerfile +jobs: + build: + name: Build Base Buildfarm Worker Image + runs-on: ubuntu-latest + steps: + - name: Checkout + uses: actions/checkout@v3 + + - name: Build Docker Image + id: buildAndPushImage + uses: MaximilianoBz/dockerhub-buildpush@v1.0 + with: + registry_url: 'docker.io' + repository_name: 'bazelbuild' + user_name: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} + password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} + image_version: 'buildfarm-worker-base:jammy' + docker_file: './ci/base-worker-image/jammy' + - name: Get pre step result output image_pull_url + run: echo "The time was ${{ steps.buildAndPushImage.outputs.image_pull_url }}" From b7daba3e2ee9efe77cd13e91aae1bfd63e89e85b Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Sat, 11 Nov 2023 16:27:24 -0500 Subject: [PATCH 53/68] Add base image generation scripts (#1532) --- ci/base-worker-image/jammy/Dockerfile | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 ci/base-worker-image/jammy/Dockerfile diff --git a/ci/base-worker-image/jammy/Dockerfile b/ci/base-worker-image/jammy/Dockerfile new file mode 100644 index 000000000..2e54a2804 --- /dev/null +++ b/ci/base-worker-image/jammy/Dockerfile @@ -0,0 +1,6 @@ +# A minimal container for building a base worker image. +# Buildfarm public releases are build using this image as a starting point. +FROM ubuntu:22.04 + +RUN apt-get update +RUN apt-get -y install default-jre default-jdk build-essential libfuse2 From dcff4f006805ff4c9664f70da9a08675b0883ffb Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Sat, 11 Nov 2023 18:55:16 -0500 Subject: [PATCH 54/68] Fix buildfarm-worker-base-build-and-deploy.yml (#1541) --- .../workflows/buildfarm-worker-base-build-and-deploy.yml | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml index 7550639dd..a91f6c859 100644 --- a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml +++ b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml @@ -19,10 +19,8 @@ jobs: uses: MaximilianoBz/dockerhub-buildpush@v1.0 with: registry_url: 'docker.io' - repository_name: 'bazelbuild' + repository_name: 'bazelbuild/buildfarm-worker-base' user_name: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} - image_version: 'buildfarm-worker-base:jammy' + image_version: 'jammy' docker_file: './ci/base-worker-image/jammy' - - name: Get pre step result output image_pull_url - run: echo "The time was ${{ steps.buildAndPushImage.outputs.image_pull_url }}" From 52318f87c29eae3967da88ab06ec8e7b92953e2f Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Tue, 14 Nov 2023 15:32:54 -0500 Subject: [PATCH 55/68] Add public buildfarm image generation actions (#1542) --- .../buildfarm-images-build-and-deploy.yml | 30 +++++++++++++++++++ ...buildfarm-worker-base-build-and-deploy.yml | 16 ++++++++-- BUILD | 26 ++++++++++++++-- ci/base-worker-image/mantic/Dockerfile | 6 ++++ images.bzl | 20 +++++++++++-- 5 files changed, 92 insertions(+), 6 deletions(-) create mode 100644 .github/workflows/buildfarm-images-build-and-deploy.yml create mode 100644 ci/base-worker-image/mantic/Dockerfile diff --git a/.github/workflows/buildfarm-images-build-and-deploy.yml b/.github/workflows/buildfarm-images-build-and-deploy.yml new file mode 100644 index 000000000..789086aaf --- /dev/null +++ b/.github/workflows/buildfarm-images-build-and-deploy.yml @@ -0,0 +1,30 @@ +name: Build and push Buildfarm images + +on: + push: + branches: + - main + +jobs: + build: + name: Build Buildfarm Images + runs-on: ubuntu-latest + steps: + - uses: bazelbuild/setup-bazelisk@v2 + + - name: Checkout + uses: actions/checkout@v3 + + - name: Login to Bazelbuild Docker Hub + uses: docker/login-action@v3 + with: + username: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} + password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} + + - name: Build Server Image + id: buildAndPushServerImage + run: bazel run public_push_buildfarm-server + + - name: Build Worker Image + id: buildAndPushWorkerImage + run: bazel run public_push_buildfarm-worker diff --git a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml index a91f6c859..ac620042b 100644 --- a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml +++ b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml @@ -6,6 +6,7 @@ on: - main paths: - ci/base-worker-image/jammy/Dockerfile + - ci/base-worker-image/mantic/Dockerfile jobs: build: name: Build Base Buildfarm Worker Image @@ -14,8 +15,8 @@ jobs: - name: Checkout uses: actions/checkout@v3 - - name: Build Docker Image - id: buildAndPushImage + - name: Build Jammy Docker Image + id: buildAndPushJammyImage uses: MaximilianoBz/dockerhub-buildpush@v1.0 with: registry_url: 'docker.io' @@ -24,3 +25,14 @@ jobs: password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} image_version: 'jammy' docker_file: './ci/base-worker-image/jammy' + + - name: Build Mantic Docker Image + id: buildAndPushManticImage + uses: MaximilianoBz/dockerhub-buildpush@v1.0 + with: + registry_url: 'docker.io' + repository_name: 'bazelbuild/buildfarm-worker-base' + user_name: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} + password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} + image_version: 'mantic' + docker_file: './ci/base-worker-image/mantic' diff --git a/BUILD b/BUILD index eb773e1a1..b81921d3c 100644 --- a/BUILD +++ b/BUILD @@ -2,7 +2,7 @@ load("@com_github_bazelbuild_buildtools//buildifier:def.bzl", "buildifier") load("@io_bazel_rules_docker//java:image.bzl", "java_image") load("@io_bazel_rules_docker//docker/package_managers:download_pkgs.bzl", "download_pkgs") load("@io_bazel_rules_docker//docker/package_managers:install_pkgs.bzl", "install_pkgs") -load("@io_bazel_rules_docker//container:container.bzl", "container_image") +load("@io_bazel_rules_docker//container:container.bzl", "container_image", "container_push") load("@rules_oss_audit//oss_audit:java/oss_audit.bzl", "oss_audit") load("//:jvm_flags.bzl", "server_jvm_flags", "worker_jvm_flags") @@ -120,7 +120,7 @@ sh_binary( java_image( name = "buildfarm-server", args = ["/app/build_buildfarm/examples/config.minimal.yml"], - base = "@ubuntu-mantic//image", + base = "@amazon_corretto_java_image_base//image", classpath_resources = [ "//src/main/java/build/buildfarm:configs", ], @@ -195,3 +195,25 @@ oss_audit( src = "//src/main/java/build/buildfarm:buildfarm-shard-worker", tags = ["audit"], ) + +# Below targets push public docker images to bazelbuild dockerhub. + +container_push( + name = "public_push_buildfarm-server", + format = "Docker", + image = ":buildfarm-server", + registry = "index.docker.io", + repository = "bazelbuild/buildfarm-server", + tag = "latest", + tags = ["container"], +) + +container_push( + name = "public_push_buildfarm-worker", + format = "Docker", + image = ":buildfarm-shard-worker", + registry = "index.docker.io", + repository = "bazelbuild/buildfarm-worker", + tag = "latest", + tags = ["container"], +) diff --git a/ci/base-worker-image/mantic/Dockerfile b/ci/base-worker-image/mantic/Dockerfile new file mode 100644 index 000000000..3dc280233 --- /dev/null +++ b/ci/base-worker-image/mantic/Dockerfile @@ -0,0 +1,6 @@ +# A minimal container for building a base worker image. +# Buildfarm public releases are build using this image as a starting point. +FROM ubuntu:23.04 + +RUN apt-get update +RUN apt-get -y install default-jre default-jdk build-essential libfuse2 diff --git a/images.bzl b/images.bzl index 939a752ed..faa70c3ff 100644 --- a/images.bzl +++ b/images.bzl @@ -26,10 +26,26 @@ def buildfarm_images(): repository = "distroless/java", ) + # Base mantic worker image for public releases (built via github action from ci/base-worker-image/mantic/Dockerfile) container_pull( name = "ubuntu-mantic", - digest = "sha256:1419bba15470a95242e917b3abcd8981ae36707b99df5ac705e1eee4d920c51c", registry = "index.docker.io", repository = "bazelbuild/buildfarm-worker-base", - tag = "mantic-java17-gcc", + tag = "mantic", + ) + + # Base worker image for public releases (built via github action from ci/base-worker-image/jammy/Dockerfile) + container_pull( + name = "ubuntu-jammy", + registry = "index.docker.io", + repository = "bazelbuild/buildfarm-worker-base", + tag = "jammy", + ) + + # Server base image + container_pull( + name = "amazon_corretto_java_image_base", + registry = "public.ecr.aws/amazoncorretto", + repository = "amazoncorretto", + tag = "21", ) From e3433937236b8c796fcab3c9fe9874a3b34e95d8 Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Wed, 15 Nov 2023 20:27:53 -0500 Subject: [PATCH 56/68] Update base image building action (#1544) --- .../buildfarm-images-build-and-deploy.yml | 2 +- ...buildfarm-worker-base-build-and-deploy.yml | 36 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/.github/workflows/buildfarm-images-build-and-deploy.yml b/.github/workflows/buildfarm-images-build-and-deploy.yml index 789086aaf..2560eadb3 100644 --- a/.github/workflows/buildfarm-images-build-and-deploy.yml +++ b/.github/workflows/buildfarm-images-build-and-deploy.yml @@ -1,4 +1,4 @@ -name: Build and push Buildfarm images +name: Build and Push Latest Buildfarm Images on: push: diff --git a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml index ac620042b..49f1cf40e 100644 --- a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml +++ b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml @@ -1,4 +1,4 @@ -name: Build and push base Buildfarm worker image +name: Build and Push Base Buildfarm Worker Images on: push: @@ -15,24 +15,24 @@ jobs: - name: Checkout uses: actions/checkout@v3 - - name: Build Jammy Docker Image - id: buildAndPushJammyImage - uses: MaximilianoBz/dockerhub-buildpush@v1.0 + - name: Login to Bazelbuild Docker Hub + uses: docker/login-action@v3 with: - registry_url: 'docker.io' - repository_name: 'bazelbuild/buildfarm-worker-base' - user_name: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} + username: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} - image_version: 'jammy' - docker_file: './ci/base-worker-image/jammy' - - name: Build Mantic Docker Image - id: buildAndPushManticImage - uses: MaximilianoBz/dockerhub-buildpush@v1.0 + - name: Build Jammy Docker image + uses: docker/build-push-action@3b5e8027fcad23fda98b2e3ac259d8d67585f671 with: - registry_url: 'docker.io' - repository_name: 'bazelbuild/buildfarm-worker-base' - user_name: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} - password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} - image_version: 'mantic' - docker_file: './ci/base-worker-image/mantic' + context: . + file: ./ci/base-worker-image/jammy/Dockerfile + push: true + tags: bazelbuild/buildfarm-worker-base:jammy + + - name: Build Mantic Docker image + uses: docker/build-push-action@3b5e8027fcad23fda98b2e3ac259d8d67585f671 + with: + context: . + file: ./ci/base-worker-image/mantic/Dockerfile + push: true + tags: bazelbuild/buildfarm-worker-base:mantic From dae7f78eda808b745d1e0bc59cfdfb4a3ba67681 Mon Sep 17 00:00:00 2001 From: Yuriy Belenitsky Date: Wed, 15 Nov 2023 21:53:23 -0500 Subject: [PATCH 57/68] Add release image generation action (#1545) --- .bazelci/presubmit.yml | 4 +++ .../buildfarm-images-build-and-deploy.yml | 4 +-- .../buildfarm-release-build-and-deploy.yml | 29 +++++++++++++++++++ BUILD | 4 +-- 4 files changed, 37 insertions(+), 4 deletions(-) create mode 100644 .github/workflows/buildfarm-release-build-and-deploy.yml diff --git a/.bazelci/presubmit.yml b/.bazelci/presubmit.yml index e08f2f07b..dfdf37f89 100644 --- a/.bazelci/presubmit.yml +++ b/.bazelci/presubmit.yml @@ -41,6 +41,8 @@ tasks: name: "Unit Tests" build_targets: - "..." + build_flags: + - "--build_tag_filters=-container" test_flags: - "--test_tag_filters=-integration,-redis" test_targets: @@ -49,6 +51,8 @@ tasks: name: "Unit Tests" build_targets: - "..." + build_flags: + - "--build_tag_filters=-container" test_flags: - "--test_tag_filters=-integration,-redis" test_targets: diff --git a/.github/workflows/buildfarm-images-build-and-deploy.yml b/.github/workflows/buildfarm-images-build-and-deploy.yml index 2560eadb3..a7f4812c9 100644 --- a/.github/workflows/buildfarm-images-build-and-deploy.yml +++ b/.github/workflows/buildfarm-images-build-and-deploy.yml @@ -23,8 +23,8 @@ jobs: - name: Build Server Image id: buildAndPushServerImage - run: bazel run public_push_buildfarm-server + run: bazel run public_push_buildfarm-server --define release_version=latest - name: Build Worker Image id: buildAndPushWorkerImage - run: bazel run public_push_buildfarm-worker + run: bazel run public_push_buildfarm-worker --define release_version=latest diff --git a/.github/workflows/buildfarm-release-build-and-deploy.yml b/.github/workflows/buildfarm-release-build-and-deploy.yml new file mode 100644 index 000000000..31200e3eb --- /dev/null +++ b/.github/workflows/buildfarm-release-build-and-deploy.yml @@ -0,0 +1,29 @@ +name: Build and Push Buildfarm Releases + +on: + release: + types: [published] + +jobs: + build: + name: Build Buildfarm Images + runs-on: ubuntu-latest + steps: + - uses: bazelbuild/setup-bazelisk@v2 + + - name: Checkout + uses: actions/checkout@v3 + + - name: Login to Bazelbuild Docker Hub + uses: docker/login-action@v3 + with: + username: ${{ secrets.BAZELBUILD_DOCKERHUB_USERNAME }} + password: ${{ secrets.BAZELBUILD_DOCKERHUB_TOKEN }} + + - name: Build Server Image + id: buildAndPushServerImage + run: bazel run public_push_buildfarm-server --define release_version=${{ github.event.release.tag_name }} + + - name: Build Worker Image + id: buildAndPushWorkerImage + run: bazel run public_push_buildfarm-worker --define release_version=${{ github.event.release.tag_name }} diff --git a/BUILD b/BUILD index b81921d3c..c0400a606 100644 --- a/BUILD +++ b/BUILD @@ -204,7 +204,7 @@ container_push( image = ":buildfarm-server", registry = "index.docker.io", repository = "bazelbuild/buildfarm-server", - tag = "latest", + tag = "$(release_version)", tags = ["container"], ) @@ -214,6 +214,6 @@ container_push( image = ":buildfarm-shard-worker", registry = "index.docker.io", repository = "bazelbuild/buildfarm-worker", - tag = "latest", + tag = "$(release_version)", tags = ["container"], ) From b01889d86e05ce088cad6899ded21038858ccdc3 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 16 Nov 2023 16:47:18 -0500 Subject: [PATCH 58/68] Limit workflow to canonical repository (#1547) --- .github/workflows/buildfarm-images-build-and-deploy.yml | 1 + .github/workflows/buildfarm-release-build-and-deploy.yml | 1 + .github/workflows/buildfarm-worker-base-build-and-deploy.yml | 1 + 3 files changed, 3 insertions(+) diff --git a/.github/workflows/buildfarm-images-build-and-deploy.yml b/.github/workflows/buildfarm-images-build-and-deploy.yml index a7f4812c9..97936b3fe 100644 --- a/.github/workflows/buildfarm-images-build-and-deploy.yml +++ b/.github/workflows/buildfarm-images-build-and-deploy.yml @@ -7,6 +7,7 @@ on: jobs: build: + if: github.repository == 'bazelbuild/bazel-buildfarm' name: Build Buildfarm Images runs-on: ubuntu-latest steps: diff --git a/.github/workflows/buildfarm-release-build-and-deploy.yml b/.github/workflows/buildfarm-release-build-and-deploy.yml index 31200e3eb..537919bbb 100644 --- a/.github/workflows/buildfarm-release-build-and-deploy.yml +++ b/.github/workflows/buildfarm-release-build-and-deploy.yml @@ -6,6 +6,7 @@ on: jobs: build: + if: github.repository == 'bazelbuild/bazel-buildfarm' name: Build Buildfarm Images runs-on: ubuntu-latest steps: diff --git a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml index 49f1cf40e..a212e5e61 100644 --- a/.github/workflows/buildfarm-worker-base-build-and-deploy.yml +++ b/.github/workflows/buildfarm-worker-base-build-and-deploy.yml @@ -9,6 +9,7 @@ on: - ci/base-worker-image/mantic/Dockerfile jobs: build: + if: github.repository == 'bazelbuild/bazel-buildfarm' name: Build Base Buildfarm Worker Image runs-on: ubuntu-latest steps: From dcee798579773981812c90ad0280dad234194ee5 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Thu, 16 Nov 2023 16:47:43 -0500 Subject: [PATCH 59/68] Check for "cores" exec property as min-cores match (#1548) The execution platform property "cores" is detailed in documentation as specifying "min-cores" and "max-cores". Match this definition and prevent "cores" from being evaluated as a strict match with the worker provision properties (with likely rejection). --- .../worker/DequeueMatchEvaluator.java | 3 +- .../worker/DequeueMatchEvaluatorTest.java | 122 +++++++++++++----- 2 files changed, 91 insertions(+), 34 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java index 3f5815fd1..8f9d71248 100644 --- a/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java +++ b/src/main/java/build/buildfarm/worker/DequeueMatchEvaluator.java @@ -119,7 +119,8 @@ private static boolean satisfiesProperties( private static boolean satisfiesProperty( SetMultimap workerProvisions, Platform.Property property) { // validate min cores - if (property.getName().equals(ExecutionProperties.MIN_CORES)) { + if (property.getName().equals(ExecutionProperties.CORES) + || property.getName().equals(ExecutionProperties.MIN_CORES)) { if (!workerProvisions.containsKey(ExecutionProperties.CORES)) { return false; } diff --git a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java index 4d484b8c4..91f83872c 100644 --- a/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java +++ b/src/test/java/build/buildfarm/worker/DequeueMatchEvaluatorTest.java @@ -14,6 +14,10 @@ package build.buildfarm.worker; +import static build.buildfarm.common.ExecutionProperties.CORES; +import static build.buildfarm.common.ExecutionProperties.MAX_CORES; +import static build.buildfarm.common.ExecutionProperties.MIN_CORES; +import static build.buildfarm.worker.DequeueMatchEvaluator.shouldKeepOperation; import static com.google.common.truth.Truth.assertThat; import build.bazel.remote.execution.v2.Platform; @@ -58,8 +62,7 @@ public void shouldKeepOperationKeepEmptyQueueEntry() throws Exception { QueueEntry entry = QueueEntry.newBuilder().setPlatform(Platform.newBuilder()).build(); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -75,19 +78,32 @@ public void shouldKeepOperationValidMinCoresQueueEntry() throws Exception { // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); - workerProvisions.put("cores", "11"); + workerProvisions.put(CORES, "11"); - QueueEntry entry = + QueueEntry minCoresEntry = QueueEntry.newBuilder() .setPlatform( Platform.newBuilder() .addProperties( - Platform.Property.newBuilder().setName("min-cores").setValue("10"))) + Platform.Property.newBuilder().setName(MIN_CORES).setValue("10"))) .build(); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, minCoresEntry); + + // ASSERT + // the worker accepts because it has more cores than the min-cores requested + assertThat(shouldKeep).isTrue(); + + QueueEntry coresEntry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName(CORES).setValue("10"))) + .build(); + + // ACT + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, coresEntry); // ASSERT // the worker accepts because it has more cores than the min-cores requested @@ -104,19 +120,32 @@ public void shouldKeepOperationInvalidMinCoresQueueEntry() throws Exception { // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); - workerProvisions.put("cores", "10"); + workerProvisions.put(CORES, "10"); - QueueEntry entry = + QueueEntry minCoresEntry = QueueEntry.newBuilder() .setPlatform( Platform.newBuilder() .addProperties( - Platform.Property.newBuilder().setName("min-cores").setValue("11"))) + Platform.Property.newBuilder().setName(MIN_CORES).setValue("11"))) .build(); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, minCoresEntry); + + // ASSERT + // the worker rejects because it has less cores than the min-cores requested + assertThat(shouldKeep).isFalse(); + + QueueEntry coresEntry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName(CORES).setValue("11"))) + .build(); + + // ACT + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, coresEntry); // ASSERT // the worker rejects because it has less cores than the min-cores requested @@ -131,25 +160,39 @@ public void shouldKeepOperationMaxCoresDoNotInfluenceAcceptance() throws Excepti // ARRANGE SetMultimap workerProvisions = HashMultimap.create(); LocalResourceSet resourceSet = new LocalResourceSet(); - workerProvisions.put("cores", "10"); + workerProvisions.put(CORES, "10"); - QueueEntry entry = + QueueEntry minCoresEntry = QueueEntry.newBuilder() .setPlatform( Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName(MIN_CORES).setValue("10")) .addProperties( - Platform.Property.newBuilder().setName("min-cores").setValue("10")) - .addProperties( - Platform.Property.newBuilder().setName("max-cores").setValue("20"))) + Platform.Property.newBuilder().setName(MAX_CORES).setValue("20"))) .build(); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, minCoresEntry); // ASSERT // the worker accepts because it has the same cores as the min-cores requested assertThat(shouldKeep).isTrue(); + + QueueEntry coresEntry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName(CORES).setValue("10")) + .addProperties( + Platform.Property.newBuilder().setName(MAX_CORES).setValue("20"))) + .build(); + + // ACT + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, coresEntry); + + // ASSERT + // the worker accepts because it has the same cores as the cores requested + assertThat(shouldKeep).isTrue(); } // Function under test: shouldKeepOperation @@ -172,8 +215,7 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E .build(); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isFalse(); @@ -182,7 +224,7 @@ public void shouldKeepOperationUnmatchedPropertiesRejectionAcceptance() throws E configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(true); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT assertThat(shouldKeep).isTrue(); @@ -211,8 +253,7 @@ public void shouldKeepOperationClaimsResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -220,7 +261,7 @@ public void shouldKeepOperationClaimsResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(0); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -252,8 +293,7 @@ public void rejectOperationIgnoresResource() throws Exception { assertThat(resourceSet.resources.get("FOO").availablePermits()).isEqualTo(1); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because the os is not satisfied @@ -288,8 +328,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(4); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -298,7 +337,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(2); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker accepts because the resource is available. @@ -307,7 +346,7 @@ public void shouldKeepOperationClaimsMultipleResource() throws Exception { assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(0); // ACT - shouldKeep = DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -348,8 +387,7 @@ public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception assertThat(resourceSet.resources.get("BAZ").availablePermits()).isEqualTo(200); // ACT - boolean shouldKeep = - DequeueMatchEvaluator.shouldKeepOperation(workerProvisions, resourceSet, entry); + boolean shouldKeep = shouldKeepOperation(workerProvisions, resourceSet, entry); // ASSERT // the worker rejects because there are no resources left. @@ -359,4 +397,22 @@ public void shouldKeepOperationFailsToClaimSameAmountRemains() throws Exception assertThat(resourceSet.resources.get("BAR").availablePermits()).isEqualTo(100); assertThat(resourceSet.resources.get("BAZ").availablePermits()).isEqualTo(200); } + + @Test + public void shouldMatchCoresAsMinAndMax() throws Exception { + SetMultimap workerProvisions = HashMultimap.create(); + LocalResourceSet resourceSet = new LocalResourceSet(); + configs.getWorker().getDequeueMatchSettings().setAllowUnmatched(false); + + QueueEntry multicoreEntry = + QueueEntry.newBuilder() + .setPlatform( + Platform.newBuilder() + .addProperties(Platform.Property.newBuilder().setName(CORES).setValue("2")) + .build()) + .build(); + + // cores must be present from worker provisions to keep cores specified in platform + assertThat(shouldKeepOperation(workerProvisions, resourceSet, multicoreEntry)).isFalse(); + } } From 221eae91e2545f22d78311b45d90734b0caa4c84 Mon Sep 17 00:00:00 2001 From: George Gensure Date: Sat, 18 Nov 2023 22:12:43 -0500 Subject: [PATCH 60/68] Consider output_* as relative to WD (#1550) Per the REAPI spec: `The paths are relative to the working directory of the action execution.` Prefix the WorkingDirectory to paths used as OutputDirectory parameters, and verify that these are present in the layout of the directory for use. --- .../worker/shard/CFCExecFileSystem.java | 8 +++- .../worker/shard/CFCExecFileSystemTest.java | 43 +++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) create mode 100644 src/test/java/build/buildfarm/worker/shard/CFCExecFileSystemTest.java diff --git a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java index dba809fdc..15aaa1078 100644 --- a/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java +++ b/src/main/java/build/buildfarm/worker/shard/CFCExecFileSystem.java @@ -46,6 +46,7 @@ import build.buildfarm.worker.ExecDirException; import build.buildfarm.worker.ExecDirException.ViolationException; import build.buildfarm.worker.OutputDirectory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; @@ -402,7 +403,8 @@ private Set linkedDirectories( return ImmutableSet.of(); } - private OutputDirectory createOutputDirectory(Command command) { + @VisibleForTesting + static OutputDirectory createOutputDirectory(Command command) { Iterable files; Iterable dirs; if (command.getOutputPathsCount() != 0) { @@ -412,6 +414,10 @@ private OutputDirectory createOutputDirectory(Command command) { files = command.getOutputFilesList(); dirs = command.getOutputDirectoriesList(); } + if (!command.getWorkingDirectory().isEmpty()) { + files = Iterables.transform(files, file -> command.getWorkingDirectory() + "/" + file); + dirs = Iterables.transform(dirs, dir -> command.getWorkingDirectory() + "/" + dir); + } return OutputDirectory.parse(files, dirs, command.getEnvironmentVariablesList()); } diff --git a/src/test/java/build/buildfarm/worker/shard/CFCExecFileSystemTest.java b/src/test/java/build/buildfarm/worker/shard/CFCExecFileSystemTest.java new file mode 100644 index 000000000..8d7c0c9f6 --- /dev/null +++ b/src/test/java/build/buildfarm/worker/shard/CFCExecFileSystemTest.java @@ -0,0 +1,43 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package build.buildfarm.worker.shard; + +import static com.google.common.truth.Truth.assertThat; + +import build.bazel.remote.execution.v2.Command; +import build.buildfarm.worker.OutputDirectory; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class CFCExecFileSystemTest { + @Test + public void outputDirectoryWorkingDirectoryRelative() { + Command command = + Command.newBuilder() + .setWorkingDirectory("foo/bar") + .addOutputFiles("baz/quux") + .addOutputDirectories("nope") + .build(); + + // verification is actually here with checked contents below + // throws unless the directory is relative to the WorkingDirectory + OutputDirectory workingOutputDirectory = + CFCExecFileSystem.createOutputDirectory(command).getChild("foo").getChild("bar"); + assertThat(workingOutputDirectory.getChild("baz").isLeaf()).isTrue(); + assertThat(workingOutputDirectory.getChild("nope").isLeaf()).isFalse(); + } +} From 021071bace27a4ddaec593c97aa4e85a6bb83428 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Fri, 27 Oct 2023 14:48:31 -0400 Subject: [PATCH 61/68] Integrate persistent workers into shard Worker via PersistentExecutor Co-authored-by: Shane Delmore --- .../buildfarm/common/ExecutionProperties.java | 16 +- src/main/java/build/buildfarm/worker/BUILD | 4 + .../java/build/buildfarm/worker/Executor.java | 39 +++ .../build/buildfarm/worker/persistent/BUILD | 29 ++ .../worker/persistent/FileAccessUtils.java | 169 ++++++++++ .../buildfarm/worker/persistent/Keymaker.java | 89 ++++++ .../worker/persistent/PersistentExecutor.java | 242 ++++++++++++++ .../worker/persistent/ProtoCoordinator.java | 295 ++++++++++++++++++ .../worker/persistent/RequestCtx.java | 29 ++ .../worker/persistent/ResponseCtx.java | 21 ++ .../worker/persistent/WorkFilesContext.java | 62 ++++ .../worker/persistent/WorkerInputs.java | 132 ++++++++ .../resources/ExecutionPropertiesParser.java | 33 ++ .../worker/resources/ResourceLimits.java | 13 + .../java/build/buildfarm/worker/util/BUILD | 24 ++ .../buildfarm/worker/util/TreeWalker.java | 111 +++++++ 16 files changed, 1307 insertions(+), 1 deletion(-) create mode 100644 src/main/java/build/buildfarm/worker/persistent/BUILD create mode 100644 src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/Keymaker.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/RequestCtx.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java create mode 100644 src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java create mode 100644 src/main/java/build/buildfarm/worker/util/BUILD create mode 100644 src/main/java/build/buildfarm/worker/util/TreeWalker.java diff --git a/src/main/java/build/buildfarm/common/ExecutionProperties.java b/src/main/java/build/buildfarm/common/ExecutionProperties.java index ee3975d6d..198783d21 100644 --- a/src/main/java/build/buildfarm/common/ExecutionProperties.java +++ b/src/main/java/build/buildfarm/common/ExecutionProperties.java @@ -297,7 +297,21 @@ public class ExecutionProperties { /** * @field WORKER * @brief The exec_property to ensure that the action only runs on the worker name given. - * @details Useful for diagnosing woker issues by targeting builds to a specific worker. + * @details Useful for diagnosing worker issues by targeting builds to a specific worker. */ public static final String WORKER = "Worker"; + + /** + * @field PERSISTENT_WORKER_KEY + * @brief Hash of tool inputs from --experiemental_remote_mark_tool_inputs + * @details See https://github.com/bazelbuild/bazel/issues/10091 + */ + public static final String PERSISTENT_WORKER_KEY = "persistentWorkerKey"; + + /** + * @field PERSISTENT_WORKER_COMMAND + * @brief Command string to start the persistent worker + * @details See https://github.com/bazelbuild/bazel/issues/10091 + */ + public static final String PERSISTENT_WORKER_COMMAND = "persistentWorkerCommand"; } diff --git a/src/main/java/build/buildfarm/worker/BUILD b/src/main/java/build/buildfarm/worker/BUILD index e24b50d73..7352a3900 100644 --- a/src/main/java/build/buildfarm/worker/BUILD +++ b/src/main/java/build/buildfarm/worker/BUILD @@ -4,12 +4,16 @@ java_library( plugins = ["//src/main/java/build/buildfarm/common:lombok"], visibility = ["//visibility:public"], deps = [ + "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", + "//persistentworkers/src/main/java/persistent/common:persistent-common", "//src/main/java/build/buildfarm/cas", "//src/main/java/build/buildfarm/common", "//src/main/java/build/buildfarm/common/config", "//src/main/java/build/buildfarm/instance", "//src/main/java/build/buildfarm/instance/stub", + "//src/main/java/build/buildfarm/worker/persistent", "//src/main/java/build/buildfarm/worker/resources", + "//src/main/java/build/buildfarm/worker/util", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "@bazel//src/main/protobuf:execution_statistics_java_proto", "@googleapis//:google_rpc_code_java_proto", diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index f4776a6f0..764f286a7 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -36,12 +36,16 @@ import build.buildfarm.common.config.ExecutionPolicy; import build.buildfarm.common.config.ExecutionWrapper; import build.buildfarm.v1test.ExecutingOperationMetadata; +import build.buildfarm.v1test.Tree; import build.buildfarm.worker.WorkerContext.IOResource; +import build.buildfarm.worker.persistent.PersistentExecutor; +import build.buildfarm.worker.persistent.WorkFilesContext; import build.buildfarm.worker.resources.ResourceLimits; import com.github.dockerjava.api.DockerClient; import com.github.dockerjava.core.DockerClientBuilder; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.devtools.build.lib.shell.Protos.ExecutionStatistics; import com.google.longrunning.Operation; import com.google.protobuf.Any; @@ -61,10 +65,14 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.logging.Level; +import java.util.logging.Logger; import lombok.extern.java.Log; @Log class Executor { + + private static final Logger logger = Logger.getLogger(Executor.class.getName()); + private static final int INCOMPLETE_EXIT_CODE = -1; private final WorkerContext workerContext; @@ -432,11 +440,42 @@ private Code executeCommand( environment.put(environmentVariable.getKey(), environmentVariable.getValue()); } + environment.putAll(limits.extraEnvironmentVariables); + // allow debugging before an execution if (limits.debugBeforeExecution) { return ExecutionDebugger.performBeforeExecutionDebug(processBuilder, limits, resultBuilder); } + boolean usePersistentWorker = + !limits.persistentWorkerKey.isEmpty() && !limits.persistentWorkerCommand.isEmpty(); + + if (usePersistentWorker) { + logger.fine( + "usePersistentWorker; got persistentWorkerCommand of : " + + limits.persistentWorkerCommand); + + Tree execTree = workerContext.getQueuedOperation(operationContext.queueEntry).getTree(); + + WorkFilesContext filesContext = + new WorkFilesContext( + execDir, + execTree, + ImmutableList.copyOf(operationContext.command.getOutputPathsList()), + ImmutableList.copyOf(operationContext.command.getOutputFilesList()), + ImmutableList.copyOf(operationContext.command.getOutputDirectoriesList())); + + return PersistentExecutor.runOnPersistentWorker( + limits.persistentWorkerCommand, + filesContext, + operationName, + ImmutableList.copyOf(arguments), + ImmutableMap.copyOf(environment), + limits, + timeout, + resultBuilder); + } + // run the action under docker if (limits.containerSettings.enabled) { DockerClient dockerClient = DockerClientBuilder.getInstance().build(); diff --git a/src/main/java/build/buildfarm/worker/persistent/BUILD b/src/main/java/build/buildfarm/worker/persistent/BUILD new file mode 100644 index 000000000..b7e04d235 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/BUILD @@ -0,0 +1,29 @@ +java_library( + name = "persistent", + srcs = glob(["*.java"]), + visibility = ["//visibility:public"], + deps = [ + "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", + "//persistentworkers/src/main/java/persistent/common:persistent-common", + "//persistentworkers/src/main/java/persistent/common/util", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/worker/resources", + "//src/main/java/build/buildfarm/worker/util", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", + "@maven//:com_google_api_grpc_proto_google_common_protos", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:commons_io_commons_io", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_context", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_netty", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + "@maven//:io_prometheus_simpleclient", + "@maven//:org_apache_commons_commons_compress", + "@maven//:org_jetbrains_annotations", + ], +) diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java new file mode 100644 index 000000000..66f9e029e --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -0,0 +1,169 @@ +package build.buildfarm.worker.persistent; + +import static java.nio.file.StandardCopyOption.COPY_ATTRIBUTES; +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; +import java.util.logging.Logger; + +public class FileAccessUtils { + + private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); + + private static final ConcurrentHashMap chm = new ConcurrentHashMap<>(); + + // Used here for locking "files" + private static class EasyMonitor { + public EasyMonitor() {} + } + + /** + * Copies a file, creating necessary directories, replacing existing files. The resulting file is + * set to be writeable, and we throw if we cannot set that. Thread-safe against writes to the same + * path. + * + * @param from + * @param to + * @throws IOException + */ + public static void copyFile(Path from, Path to) throws IOException { + Path absTo = to.toAbsolutePath(); + logger.finer("copyFile: " + from + " to " + absTo); + if (!Files.exists(from)) { + throw new IOException("copyFile: source file doesn't exist: " + from); + } + IOException ioException = + writeFileSafe( + to, + () -> { + try { + Files.copy(from, absTo, REPLACE_EXISTING, COPY_ATTRIBUTES); + boolean writeable = absTo.toFile().setWritable(true); + if (!writeable) { + return new IOException("copyFile() could not set writeable: " + absTo); + } + return null; + } catch (IOException e) { + return e; + } + }); + if (ioException != null) { + throw ioException; + } + } + + /** + * Moves a file, creating necessary directories, replacing existing files. The resulting file is + * set to be writeable, and we throw if we cannot set that. Thread-safe against writes to the same + * path. + * + * @param from + * @param to + * @throws IOException + */ + public static void moveFile(Path from, Path to) throws IOException { + Path absTo = to.toAbsolutePath(); + logger.finer("moveFile: " + from + " to " + absTo); + if (!Files.exists(from)) { + throw new IOException("moveFile: source file doesn't exist: " + from); + } + IOException ioException = + writeFileSafe( + absTo, + () -> { + try { + Files.move(from, absTo, REPLACE_EXISTING); + boolean writeable = absTo.toFile().setWritable(true); + if (!writeable) { + return new IOException("moveFile() could not set writeable: " + absTo); + } + return null; + } catch (IOException e) { + return e; + } + }); + if (ioException != null) { + throw ioException; + } + } + + /** + * Creates a symlink, creating necessary directories. Deletes pre-existing files/links which have + * the same path as the specified link, effectively overwriting any existing files/links. + * + * @param from + * @param to + * @throws IOException + */ + public static void linkFile(Path from, Path to) throws IOException { + Path absTo = to.toAbsolutePath(); + logger.finer("linkFile: " + from + " to " + absTo); + if (!Files.exists(from)) { + throw new IOException("linkFile: source file doesn't exist: " + from); + } + IOException ioException = + writeFileSafe( + absTo, + () -> { + try { + Files.deleteIfExists(absTo); + Files.createSymbolicLink(absTo, from); + return null; + } catch (IOException e) { + return e; + } + }); + if (ioException != null) { + throw ioException; + } + } + + /** + * Deletes a file; Thread-safe against writes to the same path. + * + * @param toDelete + * @throws IOException + */ + public static void deleteFileIfExists(Path toDelete) throws IOException { + Path absTo = toDelete.toAbsolutePath(); + EasyMonitor toLock = fileLock(absTo); + synchronized (toLock) { + try { + Files.deleteIfExists(absTo); + } finally { + chm.remove(absTo); + } + } + } + + /** + * Thread-safe (not multi-process-safe) wrapper for locking paths before a write operation. + * + *

This method will create necessary parent directories. + * + *

It is up to the write operation to specify whether or not to overwrite existing files. + */ + private static IOException writeFileSafe(Path absTo, Supplier writeOp) { + EasyMonitor toLock = fileLock(absTo); + synchronized (toLock) { + try { + // If 'absTo' is a symlink, checks if its target file exists + Files.createDirectories(absTo.getParent()); + return writeOp.get(); + } catch (IOException e) { + return e; + } finally { + // Clean up to prevent too many locks. + chm.remove(absTo); + } + } + } + + private static EasyMonitor fileLock(Path writeTo) { + return chm.computeIfAbsent(writeTo, k -> new EasyMonitor()); + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java new file mode 100644 index 000000000..23cfcbf57 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java @@ -0,0 +1,89 @@ +package build.buildfarm.worker.persistent; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.Objects; +import java.util.SortedMap; +import persistent.bazel.client.PersistentWorker; +import persistent.bazel.client.WorkerKey; + +public class Keymaker { + + // Constructs a key with its worker tool input files being relative paths + public static WorkerKey make( + Path opRoot, + ImmutableList workerInitCmd, + ImmutableList workerInitArgs, + ImmutableMap workerEnv, + String executionName, + WorkerInputs workerFiles) { + boolean sandboxed = true; + boolean cancellable = false; + + Path workRoot = + calculateWorkRoot( + workerInitCmd, workerInitArgs, workerEnv, executionName, sandboxed, cancellable); + Path toolsRoot = workRoot.resolve(PersistentWorker.TOOL_INPUT_SUBDIR); + + SortedMap hashedTools = workerFilesWithHashes(workerFiles); + HashCode combinedToolsHash = workerFilesCombinedHash(toolsRoot, hashedTools); + + return new WorkerKey( + workerInitCmd, + workerInitArgs, + workerEnv, + workRoot, + executionName, + combinedToolsHash, + hashedTools, + sandboxed, + cancellable); + } + + // Hash of a subset of the WorkerKey + private static Path calculateWorkRoot( + ImmutableList workerInitCmd, + ImmutableList workerInitArgs, + ImmutableMap workerEnv, + String executionName, + boolean sandboxed, + boolean cancellable) { + int workRootId = Objects.hash(workerInitCmd, workerInitArgs, workerEnv, sandboxed, cancellable); + String workRootDirName = "work-root_" + executionName + "_" + workRootId; + return PersistentExecutor.workRootsDir.resolve(workRootDirName); + } + + private static ImmutableSortedMap workerFilesWithHashes( + WorkerInputs workerFiles) { + + ImmutableSortedMap.Builder workerFileHashBuilder = + ImmutableSortedMap.naturalOrder(); + + for (Path opPath : workerFiles.opToolInputs) { + Path relPath = workerFiles.opRoot.relativize(opPath); + + HashCode toolInputHash = HashCode.fromBytes(workerFiles.digestFor(opPath).toByteArray()); + workerFileHashBuilder.put(relPath, toolInputHash); + } + + return workerFileHashBuilder.build(); + } + + // Even though we hash the toolsRoot-resolved path, it doesn't exist yet. + private static HashCode workerFilesCombinedHash( + Path toolsRoot, SortedMap hashedTools) { + Hasher hasher = Hashing.sha256().newHasher(); + hashedTools.forEach( + (relPath, toolHash) -> { + hasher.putString(toolsRoot.resolve(relPath).toString(), StandardCharsets.UTF_8); + hasher.putBytes(toolHash.asBytes()); + }); + return hasher.hash(); + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java new file mode 100644 index 000000000..374930754 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -0,0 +1,242 @@ +package build.buildfarm.worker.persistent; + +import build.bazel.remote.execution.v2.ActionResult; +import build.buildfarm.worker.resources.ResourceLimits; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest; +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse; +import com.google.protobuf.ByteString; +import com.google.protobuf.Duration; +import com.google.rpc.Code; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; +import java.util.logging.Level; +import java.util.logging.Logger; +import java.util.stream.Collectors; +import persistent.bazel.client.WorkerKey; + +/** + * Executes an Action like Executor/DockerExecutor, writing to ActionResult. + * + *

Currently has special code for discriminating between Javac/Scalac, and other persistent + * workers. + */ +public class PersistentExecutor { + + private static final Logger logger = Logger.getLogger(PersistentExecutor.class.getName()); + + // How many workers can exist at once for a given WorkerKey + // There may be multiple WorkerKeys per mnemonic, + // e.g. if builds are run with different tool fingerprints + private static final int defaultMaxWorkersPerKey = 6; + + private static final ProtoCoordinator coordinator = + ProtoCoordinator.ofCommonsPool(getMaxWorkersPerKey()); + + static final Path workRootsDir = Paths.get("/tmp/worker/persistent/"); + + static final String PERSISTENT_WORKER_FLAG = "--persistent_worker"; + + static final String JAVABUILDER_JAR = + "external/remote_java_tools/java_tools/JavaBuilder_deploy.jar"; + + private static final String SCALAC_EXEC_NAME = "Scalac"; + private static final String JAVAC_EXEC_NAME = "JavaBuilder"; + + private static int getMaxWorkersPerKey() { + try { + return Integer.parseInt(System.getenv("BUILDFARM_MAX_WORKERS_PER_KEY")); + } catch (Exception ignored) { + logger.info( + "Could not get env var BUILDFARM_MAX_WORKERS_PER_KEY; defaulting to " + + defaultMaxWorkersPerKey); + } + return defaultMaxWorkersPerKey; + } + + /** + * 1) Parses action inputs into tool inputs and request inputs 2) Makes the WorkerKey 3) Loads the + * tool inputs if needed into the WorkerKey tool inputs dir 4) Runs the work request on its + * Coordinator, passing it the required context 5) Passes output to the resultBuilder + */ + public static Code runOnPersistentWorker( + String persistentWorkerInitCmd, + WorkFilesContext context, + String operationName, + ImmutableList argsList, + ImmutableMap envVars, + ResourceLimits limits, + Duration timeout, + ActionResult.Builder resultBuilder) + throws IOException { + + //// Pull out persistent worker start command from the overall action request + + logger.log(Level.FINE, "executeCommandOnPersistentWorker[" + operationName + "]"); + + ImmutableList initCmd = parseInitCmd(persistentWorkerInitCmd, argsList); + + String executionName = getExecutionName(argsList); + if (executionName.isEmpty()) { + logger.log(Level.SEVERE, "Invalid Argument?!: " + argsList); + return Code.INVALID_ARGUMENT; + } + + ImmutableMap env; + if (executionName.equals(JAVAC_EXEC_NAME)) { + env = ImmutableMap.of(); + } else { + env = envVars; + } + + int requestArgsIdx = initCmd.size(); + ImmutableList workerExecCmd = initCmd; + ImmutableList workerInitArgs = + ImmutableList.builder().add(PERSISTENT_WORKER_FLAG).build(); + ImmutableList requestArgs = argsList.subList(requestArgsIdx, argsList.size()); + + //// Make Key + + WorkerInputs workerFiles = WorkerInputs.from(context, requestArgs); + + Path binary = Paths.get(workerExecCmd.get(0)); + if (!workerFiles.containsTool(binary) && !binary.isAbsolute()) { + throw new IllegalArgumentException("Binary isn't a tool?! " + binary); + } + + WorkerKey key = + Keymaker.make( + context.opRoot, workerExecCmd, workerInitArgs, env, executionName, workerFiles); + + coordinator.copyToolInputsIntoWorkerToolRoot(key, workerFiles); + + //// Make request + + // Inputs should be relative paths (if they are from operation root) + ImmutableList.Builder reqInputsBuilder = ImmutableList.builder(); + + for (Map.Entry opInput : workerFiles.allInputs.entrySet()) { + Input relInput = opInput.getValue(); + Path opPath = opInput.getKey(); + if (opPath.startsWith(workerFiles.opRoot)) { + relInput = + relInput.toBuilder().setPath(workerFiles.opRoot.relativize(opPath).toString()).build(); + } + reqInputsBuilder.add(relInput); + } + ImmutableList reqInputs = reqInputsBuilder.build(); + + WorkRequest request = + WorkRequest.newBuilder() + .addAllArguments(requestArgs) + .addAllInputs(reqInputs) + .setRequestId(0) + .build(); + + RequestCtx requestCtx = new RequestCtx(request, context, workerFiles, timeout); + + //// Run request + //// Required file operations (in/out) are the responsibility of the coordinator + + logger.log(Level.FINE, "Request with key: " + key); + WorkResponse response; + String stdErr = ""; + try { + ResponseCtx fullResponse = coordinator.runRequest(key, requestCtx); + + response = fullResponse.response; + stdErr = fullResponse.errorString; + } catch (Exception e) { + + String debug = + "\n\tRequest.initCmd: " + + workerExecCmd + + "\n\tRequest.initArgs: " + + workerInitArgs + + "\n\tRequest.requestArgs: " + + request.getArgumentsList(); + String msg = "Exception while running request: " + e + debug + "\n\n"; + + logger.log(Level.SEVERE, msg); + e.printStackTrace(); + response = + WorkResponse.newBuilder() + .setOutput(msg) + .setExitCode(-1) // incomplete + .build(); + } + + //// Set results + + String responseOut = response.getOutput(); + logger.log(Level.FINE, "WorkResponse.output: " + responseOut); + + int exitCode = response.getExitCode(); + resultBuilder + .setExitCode(exitCode) + .setStdoutRaw(response.getOutputBytes()) + .setStderrRaw(ByteString.copyFrom(stdErr, StandardCharsets.UTF_8)); + + if (exitCode == 0) { + return Code.OK; + } + + if (executionName.equals("SomeOtherExec")) { + System.out.println( + "SomeOtherExec inputs: " + + ImmutableList.copyOf( + reqInputs.stream().map(Input::getPath).collect(Collectors.toList()))); + } + logger.severe( + "PersistentExecutor.runOnPersistentWorker Failed with code: " + + exitCode + + "\n" + + responseOut); + return Code.FAILED_PRECONDITION; + } + + private static ImmutableList parseInitCmd(String cmdStr, ImmutableList argsList) { + if (!cmdStr.endsWith(PERSISTENT_WORKER_FLAG)) { + throw new IllegalArgumentException( + "Persistent Worker request must contain " + + PERSISTENT_WORKER_FLAG + + "\nGot: parseInitCmd[" + + cmdStr + + "]" + + "\n" + + argsList); + } + + String cmd = + cmdStr.strip().substring(0, (cmdStr.length() - PERSISTENT_WORKER_FLAG.length()) - 1); + + ImmutableList.Builder initCmdBuilder = ImmutableList.builder(); + for (String s : argsList) { + if (cmd.length() == 0) { + break; + } + cmd = cmd.substring(s.length()).strip(); + initCmdBuilder.add(s); + } + ImmutableList initCmd = initCmdBuilder.build(); + if (!initCmd.equals(argsList.subList(0, initCmd.size()))) { + throw new IllegalArgumentException("parseInitCmd?![" + initCmd + "]" + "\n" + argsList); + } + return initCmd; + } + + private static String getExecutionName(ImmutableList argsList) { + boolean isScalac = argsList.size() > 1 && argsList.get(0).endsWith("scalac/scalac"); + if (isScalac) { + return SCALAC_EXEC_NAME; + } else if (argsList.contains(JAVABUILDER_JAR)) { + return JAVAC_EXEC_NAME; + } + return "SomeOtherExec"; + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java new file mode 100644 index 000000000..0bfe339f9 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -0,0 +1,295 @@ +package build.buildfarm.worker.persistent; + +import static persistent.bazel.client.PersistentWorker.TOOL_INPUT_SUBDIR; + +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest; +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse; +import com.google.protobuf.Duration; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Timer; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.logging.Level; +import java.util.logging.Logger; +import persistent.bazel.client.CommonsWorkerPool; +import persistent.bazel.client.PersistentWorker; +import persistent.bazel.client.WorkCoordinator; +import persistent.bazel.client.WorkerKey; +import persistent.bazel.client.WorkerSupervisor; + +/** + * Responsible for: 1) Initializing a new Worker's file environment correctly 2) pre-request + * requirements, e.g. ensuring tool input files 3) post-response requirements, i.e. putting output + * files in the right place + */ +public class ProtoCoordinator extends WorkCoordinator { + + private static final Logger logger = Logger.getLogger(ProtoCoordinator.class.getName()); + + private static final String WORKER_INIT_LOG_SUFFIX = ".initargs.log"; + + private static final ConcurrentHashMap pendingReqs = + new ConcurrentHashMap<>(); + + private static final Timer timeoutScheduler = new Timer("persistent-worker-timeout", true); + + // Synchronize writes to the tool input directory per WorkerKey + // TODO: We only need a Set of WorkerKeys to synchronize on + private static final ConcurrentHashMap toolInputSyncs = + new ConcurrentHashMap<>(); + + // Enforces locking on the same object given the same WorkerKey + private static EasyMonitor keyLock(WorkerKey key) { + return toolInputSyncs.computeIfAbsent(key, k -> new EasyMonitor()); + } + + private static class EasyMonitor { + public EasyMonitor() {} + } + + public ProtoCoordinator(CommonsWorkerPool workerPool) { + super(workerPool); + } + + public ProtoCoordinator(WorkerSupervisor supervisor, int maxWorkersPerKey) { + super(new CommonsWorkerPool(supervisor, maxWorkersPerKey)); + } + + // We copy tool inputs from the shared WorkerKey tools directory into our worker exec root, + // since there are multiple workers per key, + // and presumably there might be writes to tool inputs? + // Tool inputs which are absolute-paths (e.g. /usr/bin/...) are not affected + public static ProtoCoordinator ofCommonsPool(int maxWorkersPerKey) { + WorkerSupervisor loadToolsOnCreate = + new WorkerSupervisor() { + @Override + public PersistentWorker create(WorkerKey workerKey) throws Exception { + Path keyExecRoot = workerKey.getExecRoot(); + String workerExecDir = getUniqueSubdir(keyExecRoot); + Path workerExecRoot = keyExecRoot.resolve(workerExecDir); + copyToolsIntoWorkerExecRoot(workerKey, workerExecRoot); + + Path initArgsLogFile = workerExecRoot.resolve(workerExecDir + WORKER_INIT_LOG_SUFFIX); + if (!Files.exists(initArgsLogFile)) { + StringBuilder initArgs = new StringBuilder(); + for (String s : workerKey.getCmd()) { + initArgs.append(s); + initArgs.append("\n"); + } + for (String s : workerKey.getArgs()) { + initArgs.append(s); + initArgs.append("\n"); + } + + Files.write(initArgsLogFile, initArgs.toString().getBytes()); + } + + return new PersistentWorker(workerKey, workerExecDir); + } + }; + return new ProtoCoordinator(loadToolsOnCreate, maxWorkersPerKey); + } + + public void copyToolInputsIntoWorkerToolRoot(WorkerKey key, WorkerInputs workerFiles) + throws IOException { + EasyMonitor lock = keyLock(key); + synchronized (lock) { + try { + // Move tool inputs as needed + Path workToolRoot = key.getExecRoot().resolve(PersistentWorker.TOOL_INPUT_SUBDIR); + for (Path opToolPath : workerFiles.opToolInputs) { + Path workToolPath = workerFiles.relativizeInput(workToolRoot, opToolPath); + if (!Files.exists(workToolPath)) { + workerFiles.copyInputFile(opToolPath, workToolPath); + } + } + } finally { + toolInputSyncs.remove(key); + } + } + } + + private static String getUniqueSubdir(Path workRoot) { + String uuid = UUID.randomUUID().toString(); + while (Files.exists(workRoot.resolve(uuid))) { + uuid = UUID.randomUUID().toString(); + } + return uuid; + } + + // moveToolInputsIntoWorkerToolRoot() should have been called before this. + private static void copyToolsIntoWorkerExecRoot(WorkerKey key, Path workerExecRoot) + throws IOException { + logger.log(Level.FINE, "loadToolsIntoWorkerRoot() into: " + workerExecRoot); + + Path toolInputRoot = key.getExecRoot().resolve(TOOL_INPUT_SUBDIR); + for (Path relPath : key.getWorkerFilesWithHashes().keySet()) { + Path toolInputPath = toolInputRoot.resolve(relPath); + Path execRootPath = workerExecRoot.resolve(relPath); + + FileAccessUtils.copyFile(toolInputPath, execRootPath); + } + } + + // For now, we assume that each operation corresponds to a unique worker + @Override + public WorkRequest preWorkInit(WorkerKey key, RequestCtx request, PersistentWorker worker) + throws IOException { + + PersistentWorker pendingWorker = pendingReqs.putIfAbsent(request, worker); + if (pendingWorker != null) { + if (pendingWorker != worker) { + throw new IllegalArgumentException( + "Already have a persistent worker on the job: " + request.request); + } else { + throw new IllegalArgumentException( + "Got the same request for the same worker while it's running?!: " + request.request); + } + } + startTimeoutTimer(request); + + copyNontoolInputs(request.workerInputs, worker.getExecRoot()); + + return request.request; + } + + // After the worker has finished, we need to copy output files back to the operation directory + @Override + public ResponseCtx postWorkCleanup( + WorkResponse response, PersistentWorker worker, RequestCtx request) throws IOException { + + pendingReqs.remove(request); + + if (response == null) { + throw new RuntimeException("postWorkCleanup: WorkResponse was null!"); + } + + if (response.getExitCode() == 0) { + try { + Path workerExecRoot = worker.getExecRoot(); + moveOutputsToOperationRoot(request.filesContext, workerExecRoot); + cleanUpNontoolInputs(request.workerInputs, workerExecRoot); + } catch (IOException e) { + throw logBadCleanup(request, e); + } + } + + return new ResponseCtx(response, worker.flushStdErr()); + } + + private IOException logBadCleanup(RequestCtx request, IOException e) { + WorkFilesContext context = request.filesContext; + + StringBuilder sb = new StringBuilder(); + // Why is paths empty when files are not? + sb.append( + "Output files failure debug for request with args<" + + request.request.getArgumentsList() + + ">:\n"); + sb.append("getOutputPathsList:\n"); + sb.append(context.outputPaths); + sb.append("getOutputFilesList:\n"); + sb.append(context.outputFiles); + sb.append("getOutputDirectoriesList:\n"); + sb.append(context.outputDirectories); + logger.severe(sb.toString()); + + e.printStackTrace(); + return new IOException("Response was OK but failed on exposeOutputFiles", e); + } + + // This should replace any existing symlinks + // TODO: This should hypothetically be faster+leaner than copying inputs. + // However, it's buggy. + private void linkNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) + throws IOException { + for (Path opPath : workerInputs.allInputs.keySet()) { + if (!workerInputs.allToolInputs.contains(opPath)) { + Path execPath = workerInputs.relativizeInput(workerExecRoot, opPath); + workerInputs.linkInputFile(opPath, execPath); + } + } + } + + private void copyNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) + throws IOException { + for (Path opPath : workerInputs.allInputs.keySet()) { + if (!workerInputs.allToolInputs.contains(opPath)) { + Path execPath = workerInputs.relativizeInput(workerExecRoot, opPath); + workerInputs.copyInputFile(opPath, execPath); + } + } + } + + // Make outputs visible to the rest of Worker machinery + // see DockerExecutor::copyOutputsOutOfContainer + private void moveOutputsToOperationRoot(WorkFilesContext context, Path workerExecRoot) + throws IOException { + Path opRoot = context.opRoot; + + for (String outputDir : context.outputDirectories) { + Path outputDirPath = Paths.get(outputDir); + Files.createDirectories(outputDirPath); + } + + for (String relOutput : context.outputFiles) { + Path relPath = Paths.get(relOutput); + Path opOutputPath = opRoot.resolve(relPath); + Path execOutputPath = workerExecRoot.resolve(relPath); + + FileAccessUtils.moveFile(execOutputPath, opOutputPath); + } + } + + private void cleanUpNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) + throws IOException { + for (Path opPath : workerInputs.allInputs.keySet()) { + if (!workerInputs.allToolInputs.contains(opPath)) { + workerInputs.deleteInputFileIfExists(workerExecRoot, opPath); + } + } + } + + private void startTimeoutTimer(RequestCtx request) { + Duration timeout = request.timeout; + if (timeout != null) { + long timeoutNanos = timeout.getSeconds() * 1000000000L + timeout.getNanos(); + timeoutScheduler.schedule(new RequestTimeoutHandler(request), timeoutNanos); + } + } + + private class RequestTimeoutHandler extends TimerTask { + + private final RequestCtx request; + + private RequestTimeoutHandler(RequestCtx request) { + this.request = request; + } + + @Override + public void run() { + onTimeout(this.request, pendingReqs.get(this.request)); + } + } + + private void onTimeout(RequestCtx request, PersistentWorker worker) { + if (worker != null) { + logger.severe("Persistent Worker timed out on request: " + request.request); + try { + this.workerPool.invalidateObject(worker.getKey(), worker); + } catch (Exception e) { + logger.severe( + "Tried to invalidate worker for request:\n" + + request + + "\n\tbut got: " + + e + + "\n\nCalling worker.destroy() and moving on."); + worker.destroy(); + } + } + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java new file mode 100644 index 000000000..d4d9df85d --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java @@ -0,0 +1,29 @@ +package build.buildfarm.worker.persistent; + +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest; +import com.google.protobuf.Duration; +import persistent.common.CtxAround; + +public class RequestCtx implements CtxAround { + + public final WorkRequest request; + + public final WorkFilesContext filesContext; + + public final WorkerInputs workerInputs; + + public final Duration timeout; + + public RequestCtx( + WorkRequest request, WorkFilesContext ctx, WorkerInputs workFiles, Duration timeout) { + this.request = request; + this.filesContext = ctx; + this.workerInputs = workFiles; + this.timeout = timeout; + } + + @Override + public WorkRequest get() { + return request; + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java new file mode 100644 index 000000000..578bb7ee5 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java @@ -0,0 +1,21 @@ +package build.buildfarm.worker.persistent; + +import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse; +import persistent.common.CtxAround; + +public class ResponseCtx implements CtxAround { + + public final WorkResponse response; + + public final String errorString; + + public ResponseCtx(WorkResponse response, String errorString) { + this.response = response; + this.errorString = errorString; + } + + @Override + public WorkResponse get() { + return response; + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java new file mode 100644 index 000000000..d0517657e --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java @@ -0,0 +1,62 @@ +package build.buildfarm.worker.persistent; + +import build.buildfarm.v1test.Tree; +import build.buildfarm.worker.util.TreeWalker; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import java.nio.file.Path; + +/** POJO/data class grouping all the input/output file requirements for persistent workers */ +public class WorkFilesContext { + + public final Path opRoot; + + public final Tree execTree; + + public final ImmutableList outputPaths; + + public final ImmutableList outputFiles; + + public final ImmutableList outputDirectories; + + private final TreeWalker treeWalker; + + private ImmutableMap pathInputs = null; + + private ImmutableMap toolInputs = null; + + public WorkFilesContext( + Path opRoot, + Tree execTree, + ImmutableList outputPaths, + ImmutableList outputFiles, + ImmutableList outputDirectories) { + this.opRoot = opRoot.toAbsolutePath(); + this.execTree = execTree; + this.outputPaths = outputPaths; + this.outputFiles = outputFiles; + this.outputDirectories = outputDirectories; + + this.treeWalker = new TreeWalker(execTree); + } + + // Paths are absolute paths from the opRoot; same as the Input.getPath(); + public ImmutableMap getPathInputs() { + synchronized (this) { + if (pathInputs == null) { + pathInputs = treeWalker.getAllInputs(opRoot); + } + } + return pathInputs; + } + + public ImmutableMap getToolInputs() { + synchronized (this) { + if (toolInputs == null) { + toolInputs = treeWalker.getToolInputs(opRoot); + } + } + return toolInputs; + } +} diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java new file mode 100644 index 000000000..9fe3abe8d --- /dev/null +++ b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java @@ -0,0 +1,132 @@ +package build.buildfarm.worker.persistent; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Logger; +import persistent.common.util.Args; + +public class WorkerInputs { + + private static final Logger logger = Logger.getLogger(WorkerInputs.class.getName()); + + public final Path opRoot; + // Some tool inputs are not under opRoot + public final ImmutableSet absToolInputs; + // The Paths in these collections should all be absolute and under opRoot + public final ImmutableSet opToolInputs; + public final ImmutableMap allInputs; + + public final ImmutableSet allToolInputs; + + public WorkerInputs( + Path opRoot, + ImmutableSet absToolInputs, + ImmutableSet opToolInputs, + ImmutableMap allInputs) { + this.opRoot = opRoot; + this.absToolInputs = absToolInputs; + this.opToolInputs = opToolInputs; + this.allInputs = allInputs; + + this.allToolInputs = + ImmutableSet.builder().addAll(absToolInputs).addAll(opToolInputs).build(); + + // Currently not a concern but could be in the future + for (Path tool : opToolInputs) { + if (!allInputs.containsKey(tool)) { + String msg = "Tool not found in inputs: " + tool; + logger.severe(msg); + throw new IllegalArgumentException(msg); + } + } + } + + public boolean containsTool(Path tool) { + return allToolInputs.contains(opRoot.resolve(tool)); + } + + public Path relativizeInput(Path newRoot, Path input) { + return newRoot.resolve(opRoot.relativize(input)); + } + + public void copyInputFile(Path from, Path to) throws IOException { + checkFileIsInput("copyInputFile()", from); + FileAccessUtils.copyFile(from, to); + } + + public void moveInputFile(Path from, Path to) throws IOException { + checkFileIsInput("moveInputFile()", from); + FileAccessUtils.moveFile(from, to); + } + + public void linkInputFile(Path from, Path to) throws IOException { + checkFileIsInput("linkInputFile()", from); + FileAccessUtils.linkFile(from, to); + } + + public void deleteInputFileIfExists(Path workerExecRoot, Path opPathInput) throws IOException { + checkFileIsInput("deleteInputFile()", opPathInput); + Path execPathInput = relativizeInput(workerExecRoot, opPathInput); + FileAccessUtils.deleteFileIfExists(execPathInput); + } + + private void checkFileIsInput(String operation, Path file) { + if (!allInputs.containsKey(file)) { + throw new IllegalArgumentException(operation + " called on non-input file: " + file); + } + } + + public ByteString digestFor(Path inputPath) { + Input input = allInputs.get(inputPath); + if (input == null) { + throw new IllegalArgumentException("digestFor() called on non-input file: " + inputPath); + } + return input.getDigest(); + } + + public static WorkerInputs from(WorkFilesContext workFilesContext, List reqArgs) { + + ImmutableMap pathInputs = workFilesContext.getPathInputs(); + + ImmutableSet toolsAbsPaths = workFilesContext.getToolInputs().keySet(); + + ImmutableSet toolInputs = + ImmutableSet.copyOf( + toolsAbsPaths.stream().filter(p -> p.startsWith(workFilesContext.opRoot)).iterator()); + ImmutableSet absToolInputs = + ImmutableSet.copyOf(toolsAbsPaths.stream().filter(p -> !toolInputs.contains(p)).iterator()); + + String inputsDebugMsg = + "ParsedWorkFiles:" + + "\nallInputs: " + + pathInputs.keySet() + + "\ntoolInputs: " + + toolInputs + + "\nabsToolInputs: " + + absToolInputs; + + logger.fine(inputsDebugMsg); + + return new WorkerInputs(workFilesContext.opRoot, absToolInputs, toolInputs, pathInputs); + } + + private static List argsFiles(Path opRoot, List reqArgs) { + List files = new ArrayList<>(); + for (String a : reqArgs) { + if (Args.isArgsFile(a)) { + try { + files.add(opRoot.resolve(Paths.get(a.substring(1)))); + } catch (Exception ignored) { + } + } + } + return files; + } +} diff --git a/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java b/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java index 183c48da1..3035cdb56 100644 --- a/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java +++ b/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java @@ -68,6 +68,13 @@ public static ResourceLimits Parse(Command command) { ExecutionProperties.DEBUG_TESTS_ONLY, ExecutionPropertiesParser::storeDebugTestsOnly); parser.put(ExecutionProperties.DEBUG_TARGET, ExecutionPropertiesParser::storeDebugTarget); + parser.put( + ExecutionProperties.PERSISTENT_WORKER_KEY, + ExecutionPropertiesParser::storePersistentWorkerKey); + parser.put( + ExecutionProperties.PERSISTENT_WORKER_COMMAND, + ExecutionPropertiesParser::storePersistentWorkerCommand); + ResourceLimits limits = new ResourceLimits(); command .getPlatform() @@ -327,6 +334,32 @@ private static void storeDebugTarget(ResourceLimits limits, Property property) { describeChange(limits.description, "debug target", property.getValue(), property); } + /** + * @brief Stores persistentWorkerKey + * @details Parses and stores a String. + * @param limits Current limits to apply changes to. + * @param property The property to store. + */ + private static void storePersistentWorkerKey(ResourceLimits limits, Property property) { + limits.persistentWorkerKey = property.getValue(); + ArrayList xs = new ArrayList<>(); + xs.add("Hash of tool inputs for remote persistent workers"); + describeChange(xs, "persistentWorkerKey(hash of tool inputs)", property.getValue(), property); + } + + /** + * @brief Stores persistentWorkerCommand + * @details Parses and stores a String. + * @param limits Current limits to apply changes to. + * @param property The property to store. + */ + private static void storePersistentWorkerCommand(ResourceLimits limits, Property property) { + limits.persistentWorkerCommand = property.getValue(); + ArrayList xs = new ArrayList<>(); + xs.add("persistentWorkerCommand"); + describeChange(xs, "persistentWorkerCommand", property.getValue(), property); + } + /** * @brief Store the description of the change made. * @details Adds a debug message on the resource change. diff --git a/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java b/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java index e3ddc4485..e20f2c9c2 100644 --- a/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java +++ b/src/main/java/build/buildfarm/worker/resources/ResourceLimits.java @@ -163,4 +163,17 @@ public class ResourceLimits { * @details This can be used to debug execution behavior. */ public final ArrayList description = new ArrayList<>(); + /** + * @field persistentWorkerKey + * @brief Hash of tool inputs for remote persistent workers + * @details See https://github.com/bazelbuild/bazel/issues/10091 + */ + public String persistentWorkerKey = ""; + + /** + * @field persistentWorkerCommand + * @brief Command string to start the persistent worker + * @details See https://github.com/bazelbuild/bazel/issues/10091 + */ + public String persistentWorkerCommand = ""; } diff --git a/src/main/java/build/buildfarm/worker/util/BUILD b/src/main/java/build/buildfarm/worker/util/BUILD new file mode 100644 index 000000000..b0590b061 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/util/BUILD @@ -0,0 +1,24 @@ +java_library( + name = "util", + srcs = glob(["*.java"]), + visibility = ["//visibility:public"], + deps = [ + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/instance/stub", + "//src/main/java/build/buildfarm/worker/resources", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", + "@maven//:com_google_code_gson_gson", + "@maven//:com_google_guava_guava", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_protobuf_protobuf_java_util", + "@maven//:commons_io_commons_io", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_context", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_netty", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:io_grpc_grpc_stub", + ], +) diff --git a/src/main/java/build/buildfarm/worker/util/TreeWalker.java b/src/main/java/build/buildfarm/worker/util/TreeWalker.java new file mode 100644 index 000000000..0df03b5a7 --- /dev/null +++ b/src/main/java/build/buildfarm/worker/util/TreeWalker.java @@ -0,0 +1,111 @@ +package build.buildfarm.worker.util; + +import build.bazel.remote.execution.v2.Digest; +import build.bazel.remote.execution.v2.Directory; +import build.bazel.remote.execution.v2.FileNode; +import build.bazel.remote.execution.v2.NodeProperty; +import build.buildfarm.common.ProxyDirectoriesIndex; +import build.buildfarm.v1test.Tree; +import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; + +/** + * Organizes action Inputs into files, extracting their paths, and differentiates tool inputs (e.g. + * JavaBuilder, Scalac, etc.) + */ +public class TreeWalker { + + // See: https://github.com/bazelbuild/bazel/issues/10091 + public static final String BAZEL_TOOL_INPUT_MARKER = "bazel_tool_input"; + + final Tree tree; + final Map proxyDirs; + + ImmutableMap files = null; + ImmutableMap absPathInputs = null; + ImmutableMap toolInputs = null; + + public TreeWalker(Tree tree) { + this.tree = tree; + this.proxyDirs = new ProxyDirectoriesIndex(tree.getDirectoriesMap()); + } + + public ImmutableMap getAllInputs(Path opRoot) { + if (absPathInputs == null) { + ImmutableMap relFiles = getAllFiles(); + + ImmutableMap.Builder inputs = ImmutableMap.builder(); + for (Map.Entry pf : relFiles.entrySet()) { + Path absPath = opRoot.resolve(pf.getKey()); + inputs.put(absPath, inputFromFile(absPath, pf.getValue())); + } + absPathInputs = inputs.build(); + } + return absPathInputs; + } + + public ImmutableMap getToolInputs(Path opRoot) { + if (toolInputs == null) { + ImmutableMap relFiles = getAllFiles(); + ImmutableMap.Builder inputs = ImmutableMap.builder(); + + for (Map.Entry pf : relFiles.entrySet()) { + FileNode fn = pf.getValue(); + if (isToolInput(fn)) { + Path absPath = opRoot.resolve(pf.getKey()); + inputs.put(absPath, inputFromFile(absPath, fn)); + } + } + toolInputs = inputs.build(); + } + return toolInputs; + } + + private ImmutableMap getAllFiles() { + if (files == null) { + ImmutableMap.Builder accumulator = ImmutableMap.builder(); + Directory rootDir = proxyDirs.get(tree.getRootDigest()); + files = getFilesFromDir(Paths.get("."), rootDir, accumulator).build(); + } + return files; + } + + private Input inputFromFile(Path absPath, FileNode fileNode) { + return Input.newBuilder() + .setPath(absPath.toString()) + .setDigest(fileNode.getDigest().getHashBytes()) + .build(); + } + + private ImmutableMap.Builder getFilesFromDir( + Path dirPath, Directory dir, ImmutableMap.Builder acc) { + dir.getFilesList() + .forEach( + fileNode -> { + Path path = dirPath.resolve(fileNode.getName()).normalize(); + acc.put(path, fileNode); + }); + + // Recurse into subdirectories + dir.getDirectoriesList() + .forEach( + dirNode -> + getFilesFromDir( + dirPath.resolve(dirNode.getName()), + this.proxyDirs.get(dirNode.getDigest()), + acc)); + return acc; + } + + private static boolean isToolInput(FileNode fileNode) { + for (NodeProperty prop : fileNode.getNodeProperties().getPropertiesList()) { + if (prop.getName().equals(BAZEL_TOOL_INPUT_MARKER)) { + return true; + } + } + return false; + } +} From 6bd2ab3f60c1b4fae2f9cf07d2ddabebd9a52274 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 31 Jan 2023 19:20:06 -0500 Subject: [PATCH 62/68] slight cleanup Co-authored-by: Shane Delmore --- .../java/build/buildfarm/worker/Executor.java | 5 --- .../worker/persistent/FileAccessUtils.java | 14 ++++--- .../buildfarm/worker/persistent/Keymaker.java | 2 + .../worker/persistent/PersistentExecutor.java | 34 +++++++++------- .../worker/persistent/ProtoCoordinator.java | 39 ++++++------------- .../resources/ExecutionPropertiesParser.java | 1 - .../buildfarm/worker/util/TreeWalker.java | 2 +- 7 files changed, 43 insertions(+), 54 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index 764f286a7..e6a7438a4 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -435,11 +435,6 @@ private Code executeCommand( for (EnvironmentVariable environmentVariable : environmentVariables) { environment.put(environmentVariable.getName(), environmentVariable.getValue()); } - for (Map.Entry environmentVariable : - limits.extraEnvironmentVariables.entrySet()) { - environment.put(environmentVariable.getKey(), environmentVariable.getValue()); - } - environment.putAll(limits.extraEnvironmentVariables); // allow debugging before an execution diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java index 66f9e029e..03137e20f 100644 --- a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -10,11 +10,12 @@ import java.util.function.Supplier; import java.util.logging.Logger; +// Utility for concurrent move/copy/link of files public class FileAccessUtils { private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); - private static final ConcurrentHashMap chm = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap fileLocks = new ConcurrentHashMap<>(); // Used here for locking "files" private static class EasyMonitor { @@ -23,8 +24,8 @@ public EasyMonitor() {} /** * Copies a file, creating necessary directories, replacing existing files. The resulting file is - * set to be writeable, and we throw if we cannot set that. Thread-safe against writes to the same - * path. + * set to be writeable, and we throw if we cannot set that. Thread-safe (within a process) + * against writes to the same path. * * @param from * @param to @@ -135,7 +136,7 @@ public static void deleteFileIfExists(Path toDelete) throws IOException { try { Files.deleteIfExists(absTo); } finally { - chm.remove(absTo); + fileLocks.remove(absTo); } } } @@ -158,12 +159,13 @@ private static IOException writeFileSafe(Path absTo, Supplier write return e; } finally { // Clean up to prevent too many locks. - chm.remove(absTo); + fileLocks.remove(absTo); } } } + // "Logical" file lock private static EasyMonitor fileLock(Path writeTo) { - return chm.computeIfAbsent(writeTo, k -> new EasyMonitor()); + return fileLocks.computeIfAbsent(writeTo, k -> new EasyMonitor()); } } diff --git a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java index 23cfcbf57..9663155cc 100644 --- a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java +++ b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java @@ -23,6 +23,8 @@ public static WorkerKey make( ImmutableMap workerEnv, String executionName, WorkerInputs workerFiles) { + // Cancellation not yet supported; can change in the future, + // Presumably, following how Bazel's own persistent workers work boolean sandboxed = true; boolean cancellable = false; diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java index 374930754..0cec5eead 100644 --- a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -38,10 +38,12 @@ public class PersistentExecutor { private static final ProtoCoordinator coordinator = ProtoCoordinator.ofCommonsPool(getMaxWorkersPerKey()); + // TODO load from config (i.e. {worker_root}/persistent) static final Path workRootsDir = Paths.get("/tmp/worker/persistent/"); static final String PERSISTENT_WORKER_FLAG = "--persistent_worker"; + // TODO Revisit hardcoded actions static final String JAVABUILDER_JAR = "external/remote_java_tools/java_tools/JavaBuilder_deploy.jar"; @@ -60,9 +62,12 @@ private static int getMaxWorkersPerKey() { } /** - * 1) Parses action inputs into tool inputs and request inputs 2) Makes the WorkerKey 3) Loads the - * tool inputs if needed into the WorkerKey tool inputs dir 4) Runs the work request on its - * Coordinator, passing it the required context 5) Passes output to the resultBuilder + * 1) Parses action inputs into tool inputs and request inputs + * 2) Makes the WorkerKey + * 3) Loads the tool inputs, if needed, into the WorkerKey tool inputs dir + * 4) Runs the work request on its + * Coordinator, passing it the required context + * 5) Passes output to the resultBuilder */ public static Code runOnPersistentWorker( String persistentWorkerInitCmd, @@ -83,7 +88,7 @@ public static Code runOnPersistentWorker( String executionName = getExecutionName(argsList); if (executionName.isEmpty()) { - logger.log(Level.SEVERE, "Invalid Argument?!: " + argsList); + logger.log(Level.SEVERE, "Invalid Argument: " + argsList); return Code.INVALID_ARGUMENT; } @@ -106,7 +111,7 @@ public static Code runOnPersistentWorker( Path binary = Paths.get(workerExecCmd.get(0)); if (!workerFiles.containsTool(binary) && !binary.isAbsolute()) { - throw new IllegalArgumentException("Binary isn't a tool?! " + binary); + throw new IllegalArgumentException("Binary wasn't a tool input nor an absolute path: " + binary); } WorkerKey key = @@ -185,18 +190,19 @@ public static Code runOnPersistentWorker( if (exitCode == 0) { return Code.OK; } - - if (executionName.equals("SomeOtherExec")) { - System.out.println( - "SomeOtherExec inputs: " - + ImmutableList.copyOf( - reqInputs.stream().map(Input::getPath).collect(Collectors.toList()))); - } + logger.severe( "PersistentExecutor.runOnPersistentWorker Failed with code: " + exitCode + "\n" - + responseOut); + + responseOut + + "\n" + + executionName + + " inputs:\n" + + ImmutableList.copyOf( + reqInputs.stream().map(Input::getPath).collect(Collectors.toList()) + ) + ); return Code.FAILED_PRECONDITION; } @@ -215,6 +221,7 @@ private static ImmutableList parseInitCmd(String cmdStr, ImmutableList initCmdBuilder = ImmutableList.builder(); for (String s : argsList) { if (cmd.length() == 0) { @@ -224,6 +231,7 @@ private static ImmutableList parseInitCmd(String cmdStr, ImmutableList initCmd = initCmdBuilder.build(); + // Check that the persistent worker init command matches the action command if (!initCmd.equals(argsList.subList(0, initCmd.size()))) { throw new IllegalArgumentException("parseInitCmd?![" + initCmd + "]" + "\n" + argsList); } diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java index 0bfe339f9..b08eb2e4c 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -38,17 +38,13 @@ public class ProtoCoordinator extends WorkCoordinator toolInputSyncs = + // TODO: We only need a Set of WorkerKeys to synchronize on, but no ConcurrentHashSet + private static final ConcurrentHashMap toolInputSyncs = new ConcurrentHashMap<>(); // Enforces locking on the same object given the same WorkerKey - private static EasyMonitor keyLock(WorkerKey key) { - return toolInputSyncs.computeIfAbsent(key, k -> new EasyMonitor()); - } - - private static class EasyMonitor { - public EasyMonitor() {} + private static WorkerKey keyLock(WorkerKey key) { + return toolInputSyncs.computeIfAbsent(key, k -> k); } public ProtoCoordinator(CommonsWorkerPool workerPool) { @@ -96,7 +92,7 @@ public PersistentWorker create(WorkerKey workerKey) throws Exception { public void copyToolInputsIntoWorkerToolRoot(WorkerKey key, WorkerInputs workerFiles) throws IOException { - EasyMonitor lock = keyLock(key); + WorkerKey lock = keyLock(key); synchronized (lock) { try { // Move tool inputs as needed @@ -121,7 +117,7 @@ private static String getUniqueSubdir(Path workRoot) { return uuid; } - // moveToolInputsIntoWorkerToolRoot() should have been called before this. + // copyToolInputsIntoWorkerToolRoot() should have been called before this. private static void copyToolsIntoWorkerExecRoot(WorkerKey key, Path workerExecRoot) throws IOException { logger.log(Level.FINE, "loadToolsIntoWorkerRoot() into: " + workerExecRoot); @@ -135,29 +131,30 @@ private static void copyToolsIntoWorkerExecRoot(WorkerKey key, Path workerExecRo } } - // For now, we assume that each operation corresponds to a unique worker @Override public WorkRequest preWorkInit(WorkerKey key, RequestCtx request, PersistentWorker worker) throws IOException { PersistentWorker pendingWorker = pendingReqs.putIfAbsent(request, worker); + // null means that this request was not in pendingReqs (the expected case) if (pendingWorker != null) { if (pendingWorker != worker) { throw new IllegalArgumentException( "Already have a persistent worker on the job: " + request.request); } else { throw new IllegalArgumentException( - "Got the same request for the same worker while it's running?!: " + request.request); + "Got the same request for the same worker while it's running: " + request.request); } } startTimeoutTimer(request); + // Symlinking should hypothetically be faster+leaner than copying inputs, but it's buggy. copyNontoolInputs(request.workerInputs, worker.getExecRoot()); return request.request; } - // After the worker has finished, we need to copy output files back to the operation directory + // After the worker has finished, output files need to be visible in the operation directory @Override public ResponseCtx postWorkCleanup( WorkResponse response, PersistentWorker worker, RequestCtx request) throws IOException { @@ -185,7 +182,6 @@ private IOException logBadCleanup(RequestCtx request, IOException e) { WorkFilesContext context = request.filesContext; StringBuilder sb = new StringBuilder(); - // Why is paths empty when files are not? sb.append( "Output files failure debug for request with args<" + request.request.getArgumentsList() @@ -199,20 +195,7 @@ private IOException logBadCleanup(RequestCtx request, IOException e) { logger.severe(sb.toString()); e.printStackTrace(); - return new IOException("Response was OK but failed on exposeOutputFiles", e); - } - - // This should replace any existing symlinks - // TODO: This should hypothetically be faster+leaner than copying inputs. - // However, it's buggy. - private void linkNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) - throws IOException { - for (Path opPath : workerInputs.allInputs.keySet()) { - if (!workerInputs.allToolInputs.contains(opPath)) { - Path execPath = workerInputs.relativizeInput(workerExecRoot, opPath); - workerInputs.linkInputFile(opPath, execPath); - } - } + return new IOException("Response was OK but failed on postWorkCleanup", e); } private void copyNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) diff --git a/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java b/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java index 3035cdb56..2c219589f 100644 --- a/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java +++ b/src/main/java/build/buildfarm/worker/resources/ExecutionPropertiesParser.java @@ -67,7 +67,6 @@ public static ResourceLimits Parse(Command command) { parser.put( ExecutionProperties.DEBUG_TESTS_ONLY, ExecutionPropertiesParser::storeDebugTestsOnly); parser.put(ExecutionProperties.DEBUG_TARGET, ExecutionPropertiesParser::storeDebugTarget); - parser.put( ExecutionProperties.PERSISTENT_WORKER_KEY, ExecutionPropertiesParser::storePersistentWorkerKey); diff --git a/src/main/java/build/buildfarm/worker/util/TreeWalker.java b/src/main/java/build/buildfarm/worker/util/TreeWalker.java index 0df03b5a7..4521af797 100644 --- a/src/main/java/build/buildfarm/worker/util/TreeWalker.java +++ b/src/main/java/build/buildfarm/worker/util/TreeWalker.java @@ -36,8 +36,8 @@ public TreeWalker(Tree tree) { public ImmutableMap getAllInputs(Path opRoot) { if (absPathInputs == null) { ImmutableMap relFiles = getAllFiles(); - ImmutableMap.Builder inputs = ImmutableMap.builder(); + for (Map.Entry pf : relFiles.entrySet()) { Path absPath = opRoot.resolve(pf.getKey()); inputs.put(absPath, inputFromFile(absPath, pf.getValue())); From 4b748d4d739149f20e41176a1b19f5cb9a5de748 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 31 Jan 2023 20:36:18 -0500 Subject: [PATCH 63/68] Formatting --- .../java/build/buildfarm/worker/Executor.java | 1 - .../worker/persistent/FileAccessUtils.java | 5 ++--- .../buildfarm/worker/persistent/Keymaker.java | 2 -- .../worker/persistent/PersistentExecutor.java | 21 +++++++------------ .../worker/persistent/ProtoCoordinator.java | 4 ---- .../worker/persistent/RequestCtx.java | 1 - .../worker/persistent/ResponseCtx.java | 1 - .../worker/persistent/WorkFilesContext.java | 1 - .../worker/persistent/WorkerInputs.java | 2 -- .../buildfarm/worker/util/TreeWalker.java | 3 +-- 10 files changed, 10 insertions(+), 31 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index e6a7438a4..78517cdae 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -70,7 +70,6 @@ @Log class Executor { - private static final Logger logger = Logger.getLogger(Executor.class.getName()); private static final int INCOMPLETE_EXIT_CODE = -1; diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java index 03137e20f..9711da9bf 100644 --- a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -12,7 +12,6 @@ // Utility for concurrent move/copy/link of files public class FileAccessUtils { - private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); private static final ConcurrentHashMap fileLocks = new ConcurrentHashMap<>(); @@ -24,8 +23,8 @@ public EasyMonitor() {} /** * Copies a file, creating necessary directories, replacing existing files. The resulting file is - * set to be writeable, and we throw if we cannot set that. Thread-safe (within a process) - * against writes to the same path. + * set to be writeable, and we throw if we cannot set that. Thread-safe (within a process) against + * writes to the same path. * * @param from * @param to diff --git a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java index 9663155cc..36cd5ffef 100644 --- a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java +++ b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java @@ -14,7 +14,6 @@ import persistent.bazel.client.WorkerKey; public class Keymaker { - // Constructs a key with its worker tool input files being relative paths public static WorkerKey make( Path opRoot, @@ -63,7 +62,6 @@ private static Path calculateWorkRoot( private static ImmutableSortedMap workerFilesWithHashes( WorkerInputs workerFiles) { - ImmutableSortedMap.Builder workerFileHashBuilder = ImmutableSortedMap.naturalOrder(); diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java index 0cec5eead..651eca957 100644 --- a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -27,7 +27,6 @@ * workers. */ public class PersistentExecutor { - private static final Logger logger = Logger.getLogger(PersistentExecutor.class.getName()); // How many workers can exist at once for a given WorkerKey @@ -62,12 +61,9 @@ private static int getMaxWorkersPerKey() { } /** - * 1) Parses action inputs into tool inputs and request inputs - * 2) Makes the WorkerKey - * 3) Loads the tool inputs, if needed, into the WorkerKey tool inputs dir - * 4) Runs the work request on its - * Coordinator, passing it the required context - * 5) Passes output to the resultBuilder + * 1) Parses action inputs into tool inputs and request inputs 2) Makes the WorkerKey 3) Loads the + * tool inputs, if needed, into the WorkerKey tool inputs dir 4) Runs the work request on its + * Coordinator, passing it the required context 5) Passes output to the resultBuilder */ public static Code runOnPersistentWorker( String persistentWorkerInitCmd, @@ -79,7 +75,6 @@ public static Code runOnPersistentWorker( Duration timeout, ActionResult.Builder resultBuilder) throws IOException { - //// Pull out persistent worker start command from the overall action request logger.log(Level.FINE, "executeCommandOnPersistentWorker[" + operationName + "]"); @@ -111,7 +106,8 @@ public static Code runOnPersistentWorker( Path binary = Paths.get(workerExecCmd.get(0)); if (!workerFiles.containsTool(binary) && !binary.isAbsolute()) { - throw new IllegalArgumentException("Binary wasn't a tool input nor an absolute path: " + binary); + throw new IllegalArgumentException( + "Binary wasn't a tool input nor an absolute path: " + binary); } WorkerKey key = @@ -157,7 +153,6 @@ public static Code runOnPersistentWorker( response = fullResponse.response; stdErr = fullResponse.errorString; } catch (Exception e) { - String debug = "\n\tRequest.initCmd: " + workerExecCmd @@ -190,7 +185,7 @@ public static Code runOnPersistentWorker( if (exitCode == 0) { return Code.OK; } - + logger.severe( "PersistentExecutor.runOnPersistentWorker Failed with code: " + exitCode @@ -200,9 +195,7 @@ public static Code runOnPersistentWorker( + executionName + " inputs:\n" + ImmutableList.copyOf( - reqInputs.stream().map(Input::getPath).collect(Collectors.toList()) - ) - ); + reqInputs.stream().map(Input::getPath).collect(Collectors.toList()))); return Code.FAILED_PRECONDITION; } diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java index b08eb2e4c..82943c0e6 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -27,7 +27,6 @@ * files in the right place */ public class ProtoCoordinator extends WorkCoordinator { - private static final Logger logger = Logger.getLogger(ProtoCoordinator.class.getName()); private static final String WORKER_INIT_LOG_SUFFIX = ".initargs.log"; @@ -134,7 +133,6 @@ private static void copyToolsIntoWorkerExecRoot(WorkerKey key, Path workerExecRo @Override public WorkRequest preWorkInit(WorkerKey key, RequestCtx request, PersistentWorker worker) throws IOException { - PersistentWorker pendingWorker = pendingReqs.putIfAbsent(request, worker); // null means that this request was not in pendingReqs (the expected case) if (pendingWorker != null) { @@ -158,7 +156,6 @@ public WorkRequest preWorkInit(WorkerKey key, RequestCtx request, PersistentWork @Override public ResponseCtx postWorkCleanup( WorkResponse response, PersistentWorker worker, RequestCtx request) throws IOException { - pendingReqs.remove(request); if (response == null) { @@ -246,7 +243,6 @@ private void startTimeoutTimer(RequestCtx request) { } private class RequestTimeoutHandler extends TimerTask { - private final RequestCtx request; private RequestTimeoutHandler(RequestCtx request) { diff --git a/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java index d4d9df85d..c051179ca 100644 --- a/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java +++ b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java @@ -5,7 +5,6 @@ import persistent.common.CtxAround; public class RequestCtx implements CtxAround { - public final WorkRequest request; public final WorkFilesContext filesContext; diff --git a/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java index 578bb7ee5..0a29b2254 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java +++ b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java @@ -4,7 +4,6 @@ import persistent.common.CtxAround; public class ResponseCtx implements CtxAround { - public final WorkResponse response; public final String errorString; diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java index d0517657e..c74ea56d7 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java @@ -9,7 +9,6 @@ /** POJO/data class grouping all the input/output file requirements for persistent workers */ public class WorkFilesContext { - public final Path opRoot; public final Tree execTree; diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java index 9fe3abe8d..9e2ff92a3 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java @@ -13,7 +13,6 @@ import persistent.common.util.Args; public class WorkerInputs { - private static final Logger logger = Logger.getLogger(WorkerInputs.class.getName()); public final Path opRoot; @@ -92,7 +91,6 @@ public ByteString digestFor(Path inputPath) { } public static WorkerInputs from(WorkFilesContext workFilesContext, List reqArgs) { - ImmutableMap pathInputs = workFilesContext.getPathInputs(); ImmutableSet toolsAbsPaths = workFilesContext.getToolInputs().keySet(); diff --git a/src/main/java/build/buildfarm/worker/util/TreeWalker.java b/src/main/java/build/buildfarm/worker/util/TreeWalker.java index 4521af797..a04bf1dd4 100644 --- a/src/main/java/build/buildfarm/worker/util/TreeWalker.java +++ b/src/main/java/build/buildfarm/worker/util/TreeWalker.java @@ -17,7 +17,6 @@ * JavaBuilder, Scalac, etc.) */ public class TreeWalker { - // See: https://github.com/bazelbuild/bazel/issues/10091 public static final String BAZEL_TOOL_INPUT_MARKER = "bazel_tool_input"; @@ -37,7 +36,7 @@ public ImmutableMap getAllInputs(Path opRoot) { if (absPathInputs == null) { ImmutableMap relFiles = getAllFiles(); ImmutableMap.Builder inputs = ImmutableMap.builder(); - + for (Map.Entry pf : relFiles.entrySet()) { Path absPath = opRoot.resolve(pf.getKey()); inputs.put(absPath, inputFromFile(absPath, pf.getValue())); From 2ac7a3e5f8417fdbd39db2e2e3613a2cf8e81c2c Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 31 Jan 2023 21:26:36 -0500 Subject: [PATCH 64/68] Fix static analysis --- .../worker/persistent/FileAccessUtils.java | 23 +++++++++++------ .../worker/persistent/ProtoCoordinator.java | 25 +++++++++---------- .../worker/persistent/WorkerInputs.java | 16 ------------ 3 files changed, 27 insertions(+), 37 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java index 9711da9bf..36baaa84e 100644 --- a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -11,14 +11,18 @@ import java.util.logging.Logger; // Utility for concurrent move/copy/link of files -public class FileAccessUtils { +public final class FileAccessUtils { + // singleton class with only static methods + private FileAccessUtils() {} + private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); - private static final ConcurrentHashMap fileLocks = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap fileLocks = new ConcurrentHashMap<>(); // Used here for locking "files" - private static class EasyMonitor { - public EasyMonitor() {} + private static class PathLock { + // Not used elsewhere + private PathLock() {} } /** @@ -130,7 +134,7 @@ public static void linkFile(Path from, Path to) throws IOException { */ public static void deleteFileIfExists(Path toDelete) throws IOException { Path absTo = toDelete.toAbsolutePath(); - EasyMonitor toLock = fileLock(absTo); + PathLock toLock = fileLock(absTo); synchronized (toLock) { try { Files.deleteIfExists(absTo); @@ -147,14 +151,17 @@ public static void deleteFileIfExists(Path toDelete) throws IOException { * *

It is up to the write operation to specify whether or not to overwrite existing files. */ + @SuppressWarnings("PMD.UnnecessaryLocalBeforeReturn") private static IOException writeFileSafe(Path absTo, Supplier writeOp) { - EasyMonitor toLock = fileLock(absTo); + PathLock toLock = fileLock(absTo); synchronized (toLock) { try { // If 'absTo' is a symlink, checks if its target file exists Files.createDirectories(absTo.getParent()); return writeOp.get(); } catch (IOException e) { + // PMD will complain about UnnecessaryLocalBeforeReturn + // In this case, it is necessary to catch the exception return e; } finally { // Clean up to prevent too many locks. @@ -164,7 +171,7 @@ private static IOException writeFileSafe(Path absTo, Supplier write } // "Logical" file lock - private static EasyMonitor fileLock(Path writeTo) { - return fileLocks.computeIfAbsent(writeTo, k -> new EasyMonitor()); + private static PathLock fileLock(Path writeTo) { + return fileLocks.computeIfAbsent(writeTo, k -> new PathLock()); } } diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java index 82943c0e6..3138b5652 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -73,11 +73,11 @@ public PersistentWorker create(WorkerKey workerKey) throws Exception { StringBuilder initArgs = new StringBuilder(); for (String s : workerKey.getCmd()) { initArgs.append(s); - initArgs.append("\n"); + initArgs.append('\n'); } for (String s : workerKey.getArgs()) { initArgs.append(s); - initArgs.append("\n"); + initArgs.append('\n'); } Files.write(initArgsLogFile, initArgs.toString().getBytes()); @@ -178,17 +178,16 @@ public ResponseCtx postWorkCleanup( private IOException logBadCleanup(RequestCtx request, IOException e) { WorkFilesContext context = request.filesContext; - StringBuilder sb = new StringBuilder(); - sb.append( - "Output files failure debug for request with args<" - + request.request.getArgumentsList() - + ">:\n"); - sb.append("getOutputPathsList:\n"); - sb.append(context.outputPaths); - sb.append("getOutputFilesList:\n"); - sb.append(context.outputFiles); - sb.append("getOutputDirectoriesList:\n"); - sb.append(context.outputDirectories); + StringBuilder sb = new StringBuilder(122); + sb.append("Output files failure debug for request with args<") + .append(request.request.getArgumentsList()) + .append(">:\ngetOutputPathsList:\n") + .append(context.outputPaths) + .append("getOutputFilesList:\n") + .append(context.outputFiles) + .append("getOutputDirectoriesList:\n") + .append(context.outputDirectories); + logger.severe(sb.toString()); e.printStackTrace(); diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java index 9e2ff92a3..82c8aad4c 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java @@ -6,11 +6,8 @@ import com.google.protobuf.ByteString; import java.io.IOException; import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; import java.util.List; import java.util.logging.Logger; -import persistent.common.util.Args; public class WorkerInputs { private static final Logger logger = Logger.getLogger(WorkerInputs.class.getName()); @@ -114,17 +111,4 @@ public static WorkerInputs from(WorkFilesContext workFilesContext, List return new WorkerInputs(workFilesContext.opRoot, absToolInputs, toolInputs, pathInputs); } - - private static List argsFiles(Path opRoot, List reqArgs) { - List files = new ArrayList<>(); - for (String a : reqArgs) { - if (Args.isArgsFile(a)) { - try { - files.add(opRoot.resolve(Paths.get(a.substring(1)))); - } catch (Exception ignored) { - } - } - } - return files; - } } From 957c4b59ca912396193a48dbf170599a5c85e1bb Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 31 Jan 2023 21:47:08 -0500 Subject: [PATCH 65/68] String#strip() -> String#trim() --- .../build/buildfarm/worker/persistent/PersistentExecutor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java index 651eca957..2f8cb9b61 100644 --- a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -212,7 +212,7 @@ private static ImmutableList parseInitCmd(String cmdStr, ImmutableList initCmdBuilder = ImmutableList.builder(); @@ -220,7 +220,7 @@ private static ImmutableList parseInitCmd(String cmdStr, ImmutableList initCmd = initCmdBuilder.build(); From 9ff31347ce326c20361d85d677b7bea876d28af1 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 17 Oct 2023 16:57:55 -0400 Subject: [PATCH 66/68] Rename and add tests for util/InputIndexer and init PersistentExecutorTest Co-authored-by: Shane Delmore --- .../worker/persistent/WorkFilesContext.java | 10 +- .../{TreeWalker.java => InputsIndexer.java} | 6 +- .../build/buildfarm/worker/persistent/BUILD | 31 ++++ .../persistent/PersistentExecutorTest.java | 13 ++ .../java/build/buildfarm/worker/util/BUILD | 30 ++++ .../worker/util/InputsIndexerTest.java | 163 ++++++++++++++++++ 6 files changed, 246 insertions(+), 7 deletions(-) rename src/main/java/build/buildfarm/worker/util/{TreeWalker.java => InputsIndexer.java} (96%) create mode 100644 src/test/java/build/buildfarm/worker/persistent/BUILD create mode 100644 src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java create mode 100644 src/test/java/build/buildfarm/worker/util/BUILD create mode 100644 src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java index c74ea56d7..6a4de9d04 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java @@ -1,7 +1,7 @@ package build.buildfarm.worker.persistent; import build.buildfarm.v1test.Tree; -import build.buildfarm.worker.util.TreeWalker; +import build.buildfarm.worker.util.InputsIndexer; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.devtools.build.lib.worker.WorkerProtocol.Input; @@ -19,7 +19,7 @@ public class WorkFilesContext { public final ImmutableList outputDirectories; - private final TreeWalker treeWalker; + private final InputsIndexer inputsIndexer; private ImmutableMap pathInputs = null; @@ -37,14 +37,14 @@ public WorkFilesContext( this.outputFiles = outputFiles; this.outputDirectories = outputDirectories; - this.treeWalker = new TreeWalker(execTree); + this.inputsIndexer = new InputsIndexer(execTree); } // Paths are absolute paths from the opRoot; same as the Input.getPath(); public ImmutableMap getPathInputs() { synchronized (this) { if (pathInputs == null) { - pathInputs = treeWalker.getAllInputs(opRoot); + pathInputs = inputsIndexer.getAllInputs(opRoot); } } return pathInputs; @@ -53,7 +53,7 @@ public ImmutableMap getPathInputs() { public ImmutableMap getToolInputs() { synchronized (this) { if (toolInputs == null) { - toolInputs = treeWalker.getToolInputs(opRoot); + toolInputs = inputsIndexer.getToolInputs(opRoot); } } return toolInputs; diff --git a/src/main/java/build/buildfarm/worker/util/TreeWalker.java b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java similarity index 96% rename from src/main/java/build/buildfarm/worker/util/TreeWalker.java rename to src/main/java/build/buildfarm/worker/util/InputsIndexer.java index a04bf1dd4..21d84d34b 100644 --- a/src/main/java/build/buildfarm/worker/util/TreeWalker.java +++ b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java @@ -15,8 +15,10 @@ /** * Organizes action Inputs into files, extracting their paths, and differentiates tool inputs (e.g. * JavaBuilder, Scalac, etc.) + * + *

Indexes (and partitions) Inputs from an action's Merkle Tree. */ -public class TreeWalker { +public class InputsIndexer { // See: https://github.com/bazelbuild/bazel/issues/10091 public static final String BAZEL_TOOL_INPUT_MARKER = "bazel_tool_input"; @@ -27,7 +29,7 @@ public class TreeWalker { ImmutableMap absPathInputs = null; ImmutableMap toolInputs = null; - public TreeWalker(Tree tree) { + public InputsIndexer(Tree tree) { this.tree = tree; this.proxyDirs = new ProxyDirectoriesIndex(tree.getDirectoriesMap()); } diff --git a/src/test/java/build/buildfarm/worker/persistent/BUILD b/src/test/java/build/buildfarm/worker/persistent/BUILD new file mode 100644 index 000000000..7d753777c --- /dev/null +++ b/src/test/java/build/buildfarm/worker/persistent/BUILD @@ -0,0 +1,31 @@ +java_test( + name = "tests", + size = "small", + srcs = glob(["*.java"]), + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/instance", + "//src/main/java/build/buildfarm/worker", + "//src/main/java/build/buildfarm/worker/resources", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@maven//:com_github_jnr_jnr_constants", + "@maven//:com_github_jnr_jnr_ffi", + "@maven//:com_github_serceman_jnr_fuse", + "@maven//:com_google_guava_guava", + "@maven//:com_google_jimfs_jimfs", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_context", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:org_mockito_mockito_core", + "@maven//:org_projectlombok_lombok", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) diff --git a/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java b/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java new file mode 100644 index 000000000..2d116cb90 --- /dev/null +++ b/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java @@ -0,0 +1,13 @@ +package build.buildfarm.worker.persistent; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class PersistentExecutorTest { + @Test + public void testProtoCoordinatorCreatesDirs() throws Exception { + assert (true); + } +} diff --git a/src/test/java/build/buildfarm/worker/util/BUILD b/src/test/java/build/buildfarm/worker/util/BUILD new file mode 100644 index 000000000..5e671f73e --- /dev/null +++ b/src/test/java/build/buildfarm/worker/util/BUILD @@ -0,0 +1,30 @@ +java_test( + name = "tests", + size = "small", + srcs = glob(["*.java"]), + test_class = "build.buildfarm.AllTests", + deps = [ + "//src/main/java/build/buildfarm/cas", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/worker/util", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", + "@googleapis//:google_rpc_code_java_proto", + "@maven//:com_github_jnr_jnr_constants", + "@maven//:com_github_jnr_jnr_ffi", + "@maven//:com_github_serceman_jnr_fuse", + "@maven//:com_google_guava_guava", + "@maven//:com_google_jimfs_jimfs", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_context", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:org_mockito_mockito_core", + "@maven//:org_projectlombok_lombok", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) diff --git a/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java new file mode 100644 index 000000000..f90051052 --- /dev/null +++ b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java @@ -0,0 +1,163 @@ +package build.buildfarm.worker.util; + +import static build.buildfarm.worker.util.InputsIndexer.BAZEL_TOOL_INPUT_MARKER; +import static com.google.common.truth.Truth.assertThat; + +import build.bazel.remote.execution.v2.*; +import build.buildfarm.common.DigestUtil; +import build.buildfarm.v1test.Tree; +import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import com.google.protobuf.ByteString; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.stream.Collectors; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class InputsIndexerTest { + private final DigestUtil DIGEST_UTIL = new DigestUtil(DigestUtil.HashFunction.SHA256); + + @Test + public void basicEmptyTree() { + Tree emptyTree = Tree.newBuilder().build(); + InputsIndexer indexer = new InputsIndexer(emptyTree); + assertThat(indexer.tree).isEqualTo(emptyTree); + } + + @Test + public void canGetRootDir() { + Tree.Builder treeBuilder = Tree.newBuilder(); + + Directory rootDir = Directory.getDefaultInstance(); + Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); + treeBuilder.setRootDigest(rootDirDigest); + + InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); + assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); + + Path arbitraryOpRoot = Paths.get("."); + assertThat(indexer.getAllInputs(arbitraryOpRoot).size()).isEqualTo(0); + } + + @Test + public void rootDirWithFiles() { + Tree.Builder treeBuilder = Tree.newBuilder(); + + FileNode myfile = + makeFileNode("my_file", "my file contents", NodeProperties.getDefaultInstance()); + Directory rootDir = Directory.newBuilder().addFiles(myfile).build(); + Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); + treeBuilder.setRootDigest(rootDirDigest); + + InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); + assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); + + Path arbitraryOpRoot = Paths.get("asdf"); + Input myfileInput = makeInput(arbitraryOpRoot, myfile); + + ImmutableMap expectedInputs = + ImmutableMap.of(Paths.get(myfileInput.getPath()), myfileInput); + + assertThat(indexer.getAllInputs(arbitraryOpRoot)).isEqualTo(expectedInputs); + } + + @Test + public void canRecurseAndDistinguishToolInputs() { + Tree.Builder treeBuilder = Tree.newBuilder(); + + FileNode myfile = + makeFileNode("my_file", "my file contents", NodeProperties.getDefaultInstance()); + FileNode subdirfile = + makeFileNode("subdir_file", "my subdir file contents", NodeProperties.getDefaultInstance()); + FileNode toolfile = + makeFileNode( + "tool_file", + "my tool file contents", + makeNodeProperties(ImmutableMap.of(BAZEL_TOOL_INPUT_MARKER, "value doesn't matter"))); + + Directory subDir = Directory.newBuilder().addFiles(subdirfile).build(); + String subDirName = "my_sub_dir"; + Digest subDirDigest = addDirToTree(treeBuilder, subDirName, subDir); + + Directory rootDir = + Directory.newBuilder() + .addFiles(myfile) + .addFiles(toolfile) + .addDirectories(makeDirNode(subDirName, subDirDigest)) + .build(); + + Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); + treeBuilder.setRootDigest(rootDirDigest); + + InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); + assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); + assertThat(indexer.proxyDirs.size()).isEqualTo(2); + + Path arbitraryOpRoot = Paths.get("asdf"); + Input myfileInput = makeInput(arbitraryOpRoot, myfile); + Input subdirfileInput = makeInput(arbitraryOpRoot.resolve(subDirName), subdirfile); + Input toolfileInput = makeInput(arbitraryOpRoot, toolfile); + + ImmutableMap nonToolInputs = + ImmutableMap.of( + Paths.get(myfileInput.getPath()), + myfileInput, + Paths.get(subdirfileInput.getPath()), + subdirfileInput); + ImmutableMap toolInputs = + ImmutableMap.of(Paths.get(toolfileInput.getPath()), toolfileInput); + ImmutableMap allInputs = + ImmutableMap.builder().putAll(nonToolInputs).putAll(toolInputs).build(); + + assertThat(indexer.getAllInputs(arbitraryOpRoot)).isEqualTo(allInputs); + assertThat(indexer.getAllInputs(arbitraryOpRoot).size()).isEqualTo(3); + assertThat(indexer.getToolInputs(arbitraryOpRoot)).isEqualTo(toolInputs); + } + + Digest addDirToTree(Tree.Builder treeBuilder, String dirname, Directory dir) { + ByteString dirnameBytes = ByteString.copyFromUtf8(dirname); + Digest digest = DIGEST_UTIL.compute(dirnameBytes); + String hash = digest.getHash(); + treeBuilder.putDirectories(hash, dir); + return digest; + } + + FileNode makeFileNode(String filename, String content, NodeProperties nodeProperties) { + return FileNode.newBuilder() + .setName(filename) + .setDigest(DIGEST_UTIL.compute(ByteString.copyFromUtf8(content))) + .setIsExecutable(false) + .setNodeProperties(nodeProperties) + .build(); + } + + DirectoryNode makeDirNode(String dirname, Digest dirDigest) { + // Pretty sure we don't need the actual hash for our testing purposes + return DirectoryNode.newBuilder().setName(dirname).setDigest(dirDigest).build(); + } + + NodeProperties makeNodeProperties(ImmutableMap props) { + return NodeProperties.newBuilder() + .addAllProperties( + props.entrySet().stream() + .map( + kv -> + NodeProperty.newBuilder() + .setName(kv.getKey()) + .setValue(kv.getValue()) + .build()) + .collect(Collectors.toList())) + .build(); + } + + Input makeInput(Path fileDir, FileNode file) { + Path fileNodePath = fileDir.resolve(file.getName()); + return Input.newBuilder() + .setPath(fileNodePath.toString()) + .setDigest(file.getDigest().getHashBytes()) + .build(); + } +} From 9ca026e325e24d605b7cd8b0b21daaf54f359d56 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Tue, 17 Oct 2023 17:09:58 -0400 Subject: [PATCH 67/68] Add test utils and tests for ProtoCoordinator Co-authored-by: Shane Delmore --- .../common/config/DequeueMatchSettings.java | 1 - .../java/build/buildfarm/worker/Executor.java | 8 +- .../buildfarm/worker/OperationContext.java | 2 +- .../build/buildfarm/worker/persistent/BUILD | 2 +- .../worker/persistent/FileAccessUtils.java | 67 ++---- .../buildfarm/worker/persistent/Keymaker.java | 13 +- .../worker/persistent/PersistentExecutor.java | 27 ++- .../worker/persistent/ProtoCoordinator.java | 10 +- .../worker/persistent/WorkFilesContext.java | 16 +- .../worker/persistent/WorkerInputs.java | 12 +- .../java/build/buildfarm/worker/util/BUILD | 2 +- .../buildfarm/worker/util/InputsIndexer.java | 29 ++- .../instance/shard/JedisCasWorkerMapTest.java | 1 - .../build/buildfarm/worker/persistent/BUILD | 7 +- .../persistent/PersistentExecutorTest.java | 13 -- .../persistent/ProtoCoordinatorTest.java | 118 ++++++++++ .../java/build/buildfarm/worker/util/BUILD | 35 ++- .../worker/util/InputsIndexerTest.java | 35 +-- .../worker/util/WorkerTestUtils.java | 212 ++++++++++++++++++ 19 files changed, 491 insertions(+), 119 deletions(-) delete mode 100644 src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java create mode 100644 src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java create mode 100644 src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java diff --git a/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java b/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java index 57aad7783..29655e20d 100644 --- a/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java +++ b/src/main/java/build/buildfarm/common/config/DequeueMatchSettings.java @@ -9,7 +9,6 @@ @Data public class DequeueMatchSettings { - @Getter(AccessLevel.NONE) private boolean acceptEverything; // deprecated diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index 78517cdae..9d89fc428 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -452,12 +452,7 @@ private Code executeCommand( Tree execTree = workerContext.getQueuedOperation(operationContext.queueEntry).getTree(); WorkFilesContext filesContext = - new WorkFilesContext( - execDir, - execTree, - ImmutableList.copyOf(operationContext.command.getOutputPathsList()), - ImmutableList.copyOf(operationContext.command.getOutputFilesList()), - ImmutableList.copyOf(operationContext.command.getOutputDirectoriesList())); + WorkFilesContext.fromContext(execDir, execTree, operationContext.command); return PersistentExecutor.runOnPersistentWorker( limits.persistentWorkerCommand, @@ -467,6 +462,7 @@ private Code executeCommand( ImmutableMap.copyOf(environment), limits, timeout, + PersistentExecutor.defaultWorkRootsDir, resultBuilder); } diff --git a/src/main/java/build/buildfarm/worker/OperationContext.java b/src/main/java/build/buildfarm/worker/OperationContext.java index 71b197578..ef73e1bbf 100644 --- a/src/main/java/build/buildfarm/worker/OperationContext.java +++ b/src/main/java/build/buildfarm/worker/OperationContext.java @@ -22,7 +22,7 @@ import com.google.longrunning.Operation; import java.nio.file.Path; -final class OperationContext { +public final class OperationContext { final ExecuteResponse.Builder executeResponse; final Operation operation; final Poller poller; diff --git a/src/main/java/build/buildfarm/worker/persistent/BUILD b/src/main/java/build/buildfarm/worker/persistent/BUILD index b7e04d235..242bc1bfb 100644 --- a/src/main/java/build/buildfarm/worker/persistent/BUILD +++ b/src/main/java/build/buildfarm/worker/persistent/BUILD @@ -6,11 +6,11 @@ java_library( "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", "//persistentworkers/src/main/java/persistent/common:persistent-common", "//persistentworkers/src/main/java/persistent/common/util", + "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", "//src/main/java/build/buildfarm/common", "//src/main/java/build/buildfarm/worker/resources", "//src/main/java/build/buildfarm/worker/util", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", - "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", "@maven//:com_google_api_grpc_proto_google_common_protos", "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java index 36baaa84e..8410c75dd 100644 --- a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -3,23 +3,41 @@ import static java.nio.file.StandardCopyOption.COPY_ATTRIBUTES; import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; +import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.attribute.PosixFilePermission; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; import java.util.logging.Logger; -// Utility for concurrent move/copy/link of files +/** + * Utility for concurrent move/copy of files Can be extended in the future to (sym)linking if we + * need performance + */ public final class FileAccessUtils { // singleton class with only static methods private FileAccessUtils() {} private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); + public static Path addPosixOwnerWrite(Path absPath) throws IOException { + Set perms = Files.getPosixFilePermissions(absPath); + + ImmutableSet permsWithWrite = + ImmutableSet.builder() + .addAll(perms) + .add(PosixFilePermission.OWNER_WRITE) + .build(); + + return Files.setAttribute(absPath, "posix:permissions", permsWithWrite); + } + private static final ConcurrentHashMap fileLocks = new ConcurrentHashMap<>(); - // Used here for locking "files" + // Used here as a simple lock for locking "files" (paths) private static class PathLock { // Not used elsewhere private PathLock() {} @@ -46,13 +64,10 @@ public static void copyFile(Path from, Path to) throws IOException { () -> { try { Files.copy(from, absTo, REPLACE_EXISTING, COPY_ATTRIBUTES); - boolean writeable = absTo.toFile().setWritable(true); - if (!writeable) { - return new IOException("copyFile() could not set writeable: " + absTo); - } + addPosixOwnerWrite(absTo); return null; } catch (IOException e) { - return e; + return new IOException("copyFile() could not set writeable: " + absTo, e); } }); if (ioException != null) { @@ -81,44 +96,10 @@ public static void moveFile(Path from, Path to) throws IOException { () -> { try { Files.move(from, absTo, REPLACE_EXISTING); - boolean writeable = absTo.toFile().setWritable(true); - if (!writeable) { - return new IOException("moveFile() could not set writeable: " + absTo); - } - return null; - } catch (IOException e) { - return e; - } - }); - if (ioException != null) { - throw ioException; - } - } - - /** - * Creates a symlink, creating necessary directories. Deletes pre-existing files/links which have - * the same path as the specified link, effectively overwriting any existing files/links. - * - * @param from - * @param to - * @throws IOException - */ - public static void linkFile(Path from, Path to) throws IOException { - Path absTo = to.toAbsolutePath(); - logger.finer("linkFile: " + from + " to " + absTo); - if (!Files.exists(from)) { - throw new IOException("linkFile: source file doesn't exist: " + from); - } - IOException ioException = - writeFileSafe( - absTo, - () -> { - try { - Files.deleteIfExists(absTo); - Files.createSymbolicLink(absTo, from); + addPosixOwnerWrite(absTo); return null; } catch (IOException e) { - return e; + return new IOException("copyFile() could not set writeable: " + absTo, e); } }); if (ioException != null) { diff --git a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java index 36cd5ffef..fb6a861ac 100644 --- a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java +++ b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java @@ -13,10 +13,12 @@ import persistent.bazel.client.PersistentWorker; import persistent.bazel.client.WorkerKey; +/** Much of the logic (hashing) is from Bazel itself (private library/methods, i.e. WorkerKey). */ public class Keymaker { // Constructs a key with its worker tool input files being relative paths public static WorkerKey make( Path opRoot, + Path workRootsDir, ImmutableList workerInitCmd, ImmutableList workerInitArgs, ImmutableMap workerEnv, @@ -29,7 +31,13 @@ public static WorkerKey make( Path workRoot = calculateWorkRoot( - workerInitCmd, workerInitArgs, workerEnv, executionName, sandboxed, cancellable); + workRootsDir, + workerInitCmd, + workerInitArgs, + workerEnv, + executionName, + sandboxed, + cancellable); Path toolsRoot = workRoot.resolve(PersistentWorker.TOOL_INPUT_SUBDIR); SortedMap hashedTools = workerFilesWithHashes(workerFiles); @@ -49,6 +57,7 @@ public static WorkerKey make( // Hash of a subset of the WorkerKey private static Path calculateWorkRoot( + Path workRootsDir, ImmutableList workerInitCmd, ImmutableList workerInitArgs, ImmutableMap workerEnv, @@ -57,7 +66,7 @@ private static Path calculateWorkRoot( boolean cancellable) { int workRootId = Objects.hash(workerInitCmd, workerInitArgs, workerEnv, sandboxed, cancellable); String workRootDirName = "work-root_" + executionName + "_" + workRootId; - return PersistentExecutor.workRootsDir.resolve(workRootDirName); + return workRootsDir.resolve(workRootDirName); } private static ImmutableSortedMap workerFilesWithHashes( diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java index 2f8cb9b61..5486ff7de 100644 --- a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -24,23 +24,19 @@ * Executes an Action like Executor/DockerExecutor, writing to ActionResult. * *

Currently has special code for discriminating between Javac/Scalac, and other persistent - * workers. + * workers, likely for debugging purposes, but need to revisit. (Can't remember fully since it was + * so long ago!) */ public class PersistentExecutor { private static final Logger logger = Logger.getLogger(PersistentExecutor.class.getName()); - // How many workers can exist at once for a given WorkerKey - // There may be multiple WorkerKeys per mnemonic, - // e.g. if builds are run with different tool fingerprints - private static final int defaultMaxWorkersPerKey = 6; - private static final ProtoCoordinator coordinator = ProtoCoordinator.ofCommonsPool(getMaxWorkersPerKey()); // TODO load from config (i.e. {worker_root}/persistent) - static final Path workRootsDir = Paths.get("/tmp/worker/persistent/"); + public static final Path defaultWorkRootsDir = Paths.get("/tmp/worker/persistent/"); - static final String PERSISTENT_WORKER_FLAG = "--persistent_worker"; + public static final String PERSISTENT_WORKER_FLAG = "--persistent_worker"; // TODO Revisit hardcoded actions static final String JAVABUILDER_JAR = @@ -49,6 +45,11 @@ public class PersistentExecutor { private static final String SCALAC_EXEC_NAME = "Scalac"; private static final String JAVAC_EXEC_NAME = "JavaBuilder"; + // How many workers can exist at once for a given WorkerKey + // There may be multiple WorkerKeys per mnemonic, + // e.g. if builds are run with different tool fingerprints + private static final int defaultMaxWorkersPerKey = 6; + private static int getMaxWorkersPerKey() { try { return Integer.parseInt(System.getenv("BUILDFARM_MAX_WORKERS_PER_KEY")); @@ -73,6 +74,7 @@ public static Code runOnPersistentWorker( ImmutableMap envVars, ResourceLimits limits, Duration timeout, + Path workRootsDir, ActionResult.Builder resultBuilder) throws IOException { //// Pull out persistent worker start command from the overall action request @@ -87,6 +89,7 @@ public static Code runOnPersistentWorker( return Code.INVALID_ARGUMENT; } + // TODO revisit why this was necessary in the first place ImmutableMap env; if (executionName.equals(JAVAC_EXEC_NAME)) { env = ImmutableMap.of(); @@ -112,7 +115,13 @@ public static Code runOnPersistentWorker( WorkerKey key = Keymaker.make( - context.opRoot, workerExecCmd, workerInitArgs, env, executionName, workerFiles); + context.opRoot, + workRootsDir, + workerExecCmd, + workerInitArgs, + env, + executionName, + workerFiles); coordinator.copyToolInputsIntoWorkerToolRoot(key, workerFiles); diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java index 3138b5652..69b02f05f 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -206,7 +206,7 @@ private void copyNontoolInputs(WorkerInputs workerInputs, Path workerExecRoot) // Make outputs visible to the rest of Worker machinery // see DockerExecutor::copyOutputsOutOfContainer - private void moveOutputsToOperationRoot(WorkFilesContext context, Path workerExecRoot) + void moveOutputsToOperationRoot(WorkFilesContext context, Path workerExecRoot) throws IOException { Path opRoot = context.opRoot; @@ -216,9 +216,11 @@ private void moveOutputsToOperationRoot(WorkFilesContext context, Path workerExe } for (String relOutput : context.outputFiles) { - Path relPath = Paths.get(relOutput); - Path opOutputPath = opRoot.resolve(relPath); - Path execOutputPath = workerExecRoot.resolve(relPath); + System.out.println(relOutput); + Path execOutputPath = workerExecRoot.resolve(relOutput); + System.out.println(execOutputPath); + Path opOutputPath = opRoot.resolve(relOutput); + System.out.println(opOutputPath); FileAccessUtils.moveFile(execOutputPath, opOutputPath); } diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java index 6a4de9d04..67197172a 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java @@ -1,5 +1,6 @@ package build.buildfarm.worker.persistent; +import build.bazel.remote.execution.v2.Command; import build.buildfarm.v1test.Tree; import build.buildfarm.worker.util.InputsIndexer; import com.google.common.collect.ImmutableList; @@ -37,14 +38,23 @@ public WorkFilesContext( this.outputFiles = outputFiles; this.outputDirectories = outputDirectories; - this.inputsIndexer = new InputsIndexer(execTree); + this.inputsIndexer = new InputsIndexer(execTree, this.opRoot); + } + + public static WorkFilesContext fromContext(Path opRoot, Tree inputsTree, Command opCommand) { + return new WorkFilesContext( + opRoot, + inputsTree, + ImmutableList.copyOf(opCommand.getOutputPathsList()), + ImmutableList.copyOf(opCommand.getOutputFilesList()), + ImmutableList.copyOf(opCommand.getOutputDirectoriesList())); } // Paths are absolute paths from the opRoot; same as the Input.getPath(); public ImmutableMap getPathInputs() { synchronized (this) { if (pathInputs == null) { - pathInputs = inputsIndexer.getAllInputs(opRoot); + pathInputs = inputsIndexer.getAllInputs(); } } return pathInputs; @@ -53,7 +63,7 @@ public ImmutableMap getPathInputs() { public ImmutableMap getToolInputs() { synchronized (this) { if (toolInputs == null) { - toolInputs = inputsIndexer.getToolInputs(opRoot); + toolInputs = inputsIndexer.getToolInputs(); } } return toolInputs; diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java index 82c8aad4c..4c71a4aa8 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java @@ -57,16 +57,6 @@ public void copyInputFile(Path from, Path to) throws IOException { FileAccessUtils.copyFile(from, to); } - public void moveInputFile(Path from, Path to) throws IOException { - checkFileIsInput("moveInputFile()", from); - FileAccessUtils.moveFile(from, to); - } - - public void linkInputFile(Path from, Path to) throws IOException { - checkFileIsInput("linkInputFile()", from); - FileAccessUtils.linkFile(from, to); - } - public void deleteInputFileIfExists(Path workerExecRoot, Path opPathInput) throws IOException { checkFileIsInput("deleteInputFile()", opPathInput); Path execPathInput = relativizeInput(workerExecRoot, opPathInput); @@ -109,6 +99,8 @@ public static WorkerInputs from(WorkFilesContext workFilesContext, List logger.fine(inputsDebugMsg); + System.out.println(inputsDebugMsg); + return new WorkerInputs(workFilesContext.opRoot, absToolInputs, toolInputs, pathInputs); } } diff --git a/src/main/java/build/buildfarm/worker/util/BUILD b/src/main/java/build/buildfarm/worker/util/BUILD index b0590b061..a92360d73 100644 --- a/src/main/java/build/buildfarm/worker/util/BUILD +++ b/src/main/java/build/buildfarm/worker/util/BUILD @@ -3,12 +3,12 @@ java_library( srcs = glob(["*.java"]), visibility = ["//visibility:public"], deps = [ + "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", "//src/main/java/build/buildfarm/common", "//src/main/java/build/buildfarm/instance", "//src/main/java/build/buildfarm/instance/stub", "//src/main/java/build/buildfarm/worker/resources", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", - "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", "@maven//:com_google_code_gson_gson", "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java", diff --git a/src/main/java/build/buildfarm/worker/util/InputsIndexer.java b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java index 21d84d34b..39637db54 100644 --- a/src/main/java/build/buildfarm/worker/util/InputsIndexer.java +++ b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java @@ -8,8 +8,8 @@ import build.buildfarm.v1test.Tree; import com.google.common.collect.ImmutableMap; import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import java.nio.file.FileSystem; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Map; /** @@ -25,22 +25,35 @@ public class InputsIndexer { final Tree tree; final Map proxyDirs; + final FileSystem fs; + + final Path opRoot; + ImmutableMap files = null; ImmutableMap absPathInputs = null; ImmutableMap toolInputs = null; - public InputsIndexer(Tree tree) { + public InputsIndexer(Tree tree, Path opRoot) { this.tree = tree; this.proxyDirs = new ProxyDirectoriesIndex(tree.getDirectoriesMap()); + this.opRoot = opRoot; + this.fs = opRoot.getFileSystem(); } - public ImmutableMap getAllInputs(Path opRoot) { + // https://stackoverflow.com/questions/22611919/why-do-i-get-providermismatchexception-when-i-try-to-relativize-a-path-agains + public Path pathTransform(final Path path) { + Path ret = fs.getPath(path.isAbsolute() ? fs.getSeparator() : ""); + for (final Path component : path) ret = ret.resolve(component.getFileName().toString()); + return ret; + } + + public ImmutableMap getAllInputs() { if (absPathInputs == null) { ImmutableMap relFiles = getAllFiles(); ImmutableMap.Builder inputs = ImmutableMap.builder(); for (Map.Entry pf : relFiles.entrySet()) { - Path absPath = opRoot.resolve(pf.getKey()); + Path absPath = this.opRoot.resolve(pf.getKey()).normalize(); inputs.put(absPath, inputFromFile(absPath, pf.getValue())); } absPathInputs = inputs.build(); @@ -48,7 +61,7 @@ public ImmutableMap getAllInputs(Path opRoot) { return absPathInputs; } - public ImmutableMap getToolInputs(Path opRoot) { + public ImmutableMap getToolInputs() { if (toolInputs == null) { ImmutableMap relFiles = getAllFiles(); ImmutableMap.Builder inputs = ImmutableMap.builder(); @@ -56,7 +69,7 @@ public ImmutableMap getToolInputs(Path opRoot) { for (Map.Entry pf : relFiles.entrySet()) { FileNode fn = pf.getValue(); if (isToolInput(fn)) { - Path absPath = opRoot.resolve(pf.getKey()); + Path absPath = this.opRoot.resolve(pf.getKey()); inputs.put(absPath, inputFromFile(absPath, fn)); } } @@ -69,7 +82,9 @@ private ImmutableMap getAllFiles() { if (files == null) { ImmutableMap.Builder accumulator = ImmutableMap.builder(); Directory rootDir = proxyDirs.get(tree.getRootDigest()); - files = getFilesFromDir(Paths.get("."), rootDir, accumulator).build(); + + Path fsRelative = fs.getPath("."); + files = getFilesFromDir(fsRelative, rootDir, accumulator).build(); } return files; } diff --git a/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java b/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java index caa69536c..0b6f3d202 100644 --- a/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java +++ b/src/test/java/build/buildfarm/instance/shard/JedisCasWorkerMapTest.java @@ -20,7 +20,6 @@ @RunWith(JUnit4.class) public class JedisCasWorkerMapTest { - private static final String CAS_PREFIX = "ContentAddressableStorage"; private RedisServer redisServer; diff --git a/src/test/java/build/buildfarm/worker/persistent/BUILD b/src/test/java/build/buildfarm/worker/persistent/BUILD index 7d753777c..0520097ff 100644 --- a/src/test/java/build/buildfarm/worker/persistent/BUILD +++ b/src/test/java/build/buildfarm/worker/persistent/BUILD @@ -4,14 +4,19 @@ java_test( srcs = glob(["*.java"]), test_class = "build.buildfarm.AllTests", deps = [ + "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", + "//persistentworkers/src/main/java/persistent/common:persistent-common", + "//persistentworkers/src/main/java/persistent/common/util", + "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", "//src/main/java/build/buildfarm/common", "//src/main/java/build/buildfarm/common/config", "//src/main/java/build/buildfarm/instance", "//src/main/java/build/buildfarm/worker", + "//src/main/java/build/buildfarm/worker/persistent", "//src/main/java/build/buildfarm/worker/resources", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "//src/test/java/build/buildfarm:test_runner", - "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", + "//src/test/java/build/buildfarm/worker/util:worker_test_utils", "@googleapis//:google_rpc_code_java_proto", "@maven//:com_github_jnr_jnr_constants", "@maven//:com_github_jnr_jnr_ffi", diff --git a/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java b/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java deleted file mode 100644 index 2d116cb90..000000000 --- a/src/test/java/build/buildfarm/worker/persistent/PersistentExecutorTest.java +++ /dev/null @@ -1,13 +0,0 @@ -package build.buildfarm.worker.persistent; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class PersistentExecutorTest { - @Test - public void testProtoCoordinatorCreatesDirs() throws Exception { - assert (true); - } -} diff --git a/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java b/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java new file mode 100644 index 000000000..998d510cd --- /dev/null +++ b/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java @@ -0,0 +1,118 @@ +package build.buildfarm.worker.persistent; + +import build.bazel.remote.execution.v2.Command; +import build.buildfarm.v1test.Tree; +import build.buildfarm.worker.util.WorkerTestUtils; +import build.buildfarm.worker.util.WorkerTestUtils.TreeFile; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.jimfs.Configuration; +import com.google.common.jimfs.Jimfs; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import persistent.bazel.client.PersistentWorker; +import persistent.bazel.client.WorkerKey; + +@RunWith(JUnit4.class) +public class ProtoCoordinatorTest { + private WorkerKey makeWorkerKey( + WorkFilesContext ctx, WorkerInputs workerFiles, Path workRootsDir) { + return Keymaker.make( + ctx.opRoot, + workRootsDir, + ImmutableList.of("workerExecCmd"), + ImmutableList.of("workerInitArgs"), + ImmutableMap.of(), + "executionName", + workerFiles); + } + + private Path rootDir = null; + + public Path jimFsRoot() { + if (rootDir == null) { + rootDir = + Iterables.getFirst( + Jimfs.newFileSystem( + Configuration.unix() + .toBuilder() + .setAttributeViews("basic", "owner", "posix", "unix") + .build()) + .getRootDirectories(), + null); + } + return rootDir; + } + + @Test + public void testProtoCoordinator() throws Exception { + ProtoCoordinator pc = ProtoCoordinator.ofCommonsPool(4); + + Path fsRoot = jimFsRoot(); + Path opRoot = fsRoot.resolve("opRoot"); + assert (Files.notExists(opRoot)); + Files.createDirectory(opRoot); + + assert (Files.exists(opRoot)); + + String treeRootDir = opRoot.toString(); + List fileInputs = + ImmutableList.of( + new TreeFile("file_1", "file contents 1"), + new TreeFile("subdir/subdir_file_2", "file contents 2"), + new TreeFile("tools_dir/tool_file", "tool file contents", true), + new TreeFile("tools_dir/tool_file_2", "tool file contents 2", true)); + + Tree tree = WorkerTestUtils.makeTree(treeRootDir, fileInputs); + + Command command = WorkerTestUtils.makeCommand(); + WorkFilesContext ctx = WorkFilesContext.fromContext(opRoot, tree, command); + ImmutableList requestArgs = ImmutableList.of("reqArg1"); + + WorkerInputs workerFiles = WorkerInputs.from(ctx, requestArgs); + + for (Map.Entry entry : workerFiles.allInputs.entrySet()) { + Path file = entry.getKey(); + Files.createDirectories(file.getParent()); + Files.createFile(file); + } + + WorkerKey key = makeWorkerKey(ctx, workerFiles, fsRoot.resolve("workRootsDir")); + + Path workRoot = key.getExecRoot(); + Path toolsRoot = workRoot.resolve(PersistentWorker.TOOL_INPUT_SUBDIR); + + pc.copyToolInputsIntoWorkerToolRoot(key, workerFiles); + + assert Files.exists(workRoot); + List expectedToolInputs = new ArrayList<>(); + for (TreeFile file : fileInputs) { + if (file.isTool) { + expectedToolInputs.add(toolsRoot.resolve(file.path)); + } + } + WorkerTestUtils.assertFilesExistExactly(workRoot, expectedToolInputs); + + List expectedOpRootFiles = new ArrayList<>(); + + // Check that we move specified output files (assuming they exist) + for (String pathStr : ctx.outputFiles) { + Path file = workRoot.resolve(pathStr); + Files.createDirectories(file.getParent()); + Files.createFile(file); + expectedOpRootFiles.add(opRoot.resolve(pathStr)); + } + + pc.moveOutputsToOperationRoot(ctx, workRoot); + + WorkerTestUtils.assertFilesExistExactly(opRoot, expectedOpRootFiles); + } +} diff --git a/src/test/java/build/buildfarm/worker/util/BUILD b/src/test/java/build/buildfarm/worker/util/BUILD index 5e671f73e..c0d0bbe46 100644 --- a/src/test/java/build/buildfarm/worker/util/BUILD +++ b/src/test/java/build/buildfarm/worker/util/BUILD @@ -1,16 +1,47 @@ +java_library( + name = "worker_test_utils", + srcs = ["WorkerTestUtils.java"], + visibility = ["//src/test/java:__subpackages__"], + deps = [ + "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", + "//src/main/java/build/buildfarm/cas", + "//src/main/java/build/buildfarm/common", + "//src/main/java/build/buildfarm/common/config", + "//src/main/java/build/buildfarm/worker/util", + "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", + "//src/test/java/build/buildfarm:test_runner", + "@googleapis//:google_rpc_code_java_proto", + "@maven//:com_github_jnr_jnr_constants", + "@maven//:com_github_jnr_jnr_ffi", + "@maven//:com_github_serceman_jnr_fuse", + "@maven//:com_google_guava_guava", + "@maven//:com_google_jimfs_jimfs", + "@maven//:com_google_protobuf_protobuf_java", + "@maven//:com_google_truth_truth", + "@maven//:io_grpc_grpc_api", + "@maven//:io_grpc_grpc_context", + "@maven//:io_grpc_grpc_core", + "@maven//:io_grpc_grpc_protobuf", + "@maven//:org_mockito_mockito_core", + "@maven//:org_projectlombok_lombok", + "@remote_apis//:build_bazel_remote_execution_v2_remote_execution_java_proto", + ], +) + java_test( name = "tests", size = "small", - srcs = glob(["*.java"]), + srcs = glob(["*Test.java"]), test_class = "build.buildfarm.AllTests", deps = [ + ":worker_test_utils", + "//persistentworkers/src/main/protobuf:worker_protocol_java_proto", "//src/main/java/build/buildfarm/cas", "//src/main/java/build/buildfarm/common", "//src/main/java/build/buildfarm/common/config", "//src/main/java/build/buildfarm/worker/util", "//src/main/protobuf:build_buildfarm_v1test_buildfarm_java_proto", "//src/test/java/build/buildfarm:test_runner", - "@bazel_tools//src/main/protobuf:worker_protocol_java_proto", "@googleapis//:google_rpc_code_java_proto", "@maven//:com_github_jnr_jnr_constants", "@maven//:com_github_jnr_jnr_ffi", diff --git a/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java index f90051052..d81048bce 100644 --- a/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java +++ b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java @@ -3,7 +3,12 @@ import static build.buildfarm.worker.util.InputsIndexer.BAZEL_TOOL_INPUT_MARKER; import static com.google.common.truth.Truth.assertThat; -import build.bazel.remote.execution.v2.*; +import build.bazel.remote.execution.v2.Digest; +import build.bazel.remote.execution.v2.Directory; +import build.bazel.remote.execution.v2.DirectoryNode; +import build.bazel.remote.execution.v2.FileNode; +import build.bazel.remote.execution.v2.NodeProperties; +import build.bazel.remote.execution.v2.NodeProperty; import build.buildfarm.common.DigestUtil; import build.buildfarm.v1test.Tree; import com.google.common.collect.ImmutableMap; @@ -16,6 +21,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +// TODO: use WorkerTestUtils.makeTree @RunWith(JUnit4.class) public class InputsIndexerTest { private final DigestUtil DIGEST_UTIL = new DigestUtil(DigestUtil.HashFunction.SHA256); @@ -23,7 +29,7 @@ public class InputsIndexerTest { @Test public void basicEmptyTree() { Tree emptyTree = Tree.newBuilder().build(); - InputsIndexer indexer = new InputsIndexer(emptyTree); + InputsIndexer indexer = new InputsIndexer(emptyTree, Paths.get(".")); assertThat(indexer.tree).isEqualTo(emptyTree); } @@ -35,11 +41,11 @@ public void canGetRootDir() { Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); treeBuilder.setRootDigest(rootDirDigest); - InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); - assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); - Path arbitraryOpRoot = Paths.get("."); - assertThat(indexer.getAllInputs(arbitraryOpRoot).size()).isEqualTo(0); + + InputsIndexer indexer = new InputsIndexer(treeBuilder.build(), arbitraryOpRoot); + assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); + assertThat(indexer.getAllInputs().size()).isEqualTo(0); } @Test @@ -52,16 +58,16 @@ public void rootDirWithFiles() { Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); treeBuilder.setRootDigest(rootDirDigest); - InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); + Path arbitraryOpRoot = Paths.get("asdf"); + InputsIndexer indexer = new InputsIndexer(treeBuilder.build(), arbitraryOpRoot); assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); - Path arbitraryOpRoot = Paths.get("asdf"); Input myfileInput = makeInput(arbitraryOpRoot, myfile); ImmutableMap expectedInputs = ImmutableMap.of(Paths.get(myfileInput.getPath()), myfileInput); - assertThat(indexer.getAllInputs(arbitraryOpRoot)).isEqualTo(expectedInputs); + assertThat(indexer.getAllInputs()).isEqualTo(expectedInputs); } @Test @@ -92,11 +98,12 @@ public void canRecurseAndDistinguishToolInputs() { Digest rootDirDigest = addDirToTree(treeBuilder, "my_root_dir", rootDir); treeBuilder.setRootDigest(rootDirDigest); - InputsIndexer indexer = new InputsIndexer(treeBuilder.build()); + Path arbitraryOpRoot = Paths.get("asdf"); + + InputsIndexer indexer = new InputsIndexer(treeBuilder.build(), arbitraryOpRoot); assertThat(indexer.proxyDirs.get(rootDirDigest)).isEqualTo(rootDir); assertThat(indexer.proxyDirs.size()).isEqualTo(2); - Path arbitraryOpRoot = Paths.get("asdf"); Input myfileInput = makeInput(arbitraryOpRoot, myfile); Input subdirfileInput = makeInput(arbitraryOpRoot.resolve(subDirName), subdirfile); Input toolfileInput = makeInput(arbitraryOpRoot, toolfile); @@ -112,9 +119,9 @@ public void canRecurseAndDistinguishToolInputs() { ImmutableMap allInputs = ImmutableMap.builder().putAll(nonToolInputs).putAll(toolInputs).build(); - assertThat(indexer.getAllInputs(arbitraryOpRoot)).isEqualTo(allInputs); - assertThat(indexer.getAllInputs(arbitraryOpRoot).size()).isEqualTo(3); - assertThat(indexer.getToolInputs(arbitraryOpRoot)).isEqualTo(toolInputs); + assertThat(indexer.getAllInputs()).isEqualTo(allInputs); + assertThat(indexer.getAllInputs().size()).isEqualTo(3); + assertThat(indexer.getToolInputs()).isEqualTo(toolInputs); } Digest addDirToTree(Tree.Builder treeBuilder, String dirname, Directory dir) { diff --git a/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java b/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java new file mode 100644 index 000000000..1a6a687e7 --- /dev/null +++ b/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java @@ -0,0 +1,212 @@ +package build.buildfarm.worker.util; + +import static build.buildfarm.worker.util.InputsIndexer.BAZEL_TOOL_INPUT_MARKER; +import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; + +import build.bazel.remote.execution.v2.Command; +import build.bazel.remote.execution.v2.Digest; +import build.bazel.remote.execution.v2.Directory; +import build.bazel.remote.execution.v2.DirectoryNode; +import build.bazel.remote.execution.v2.FileNode; +import build.bazel.remote.execution.v2.NodeProperties; +import build.bazel.remote.execution.v2.NodeProperty; +import build.buildfarm.common.DigestUtil; +import build.buildfarm.v1test.Tree; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.devtools.build.lib.worker.WorkerProtocol.Input; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.FileVisitor; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class WorkerTestUtils { + public static final DigestUtil DIGEST_UTIL = new DigestUtil(DigestUtil.HashFunction.SHA256); + + public static FileNode makeFileNode( + String filename, String content, NodeProperties nodeProperties) { + return FileNode.newBuilder() + .setName(filename) + .setDigest(DIGEST_UTIL.compute(ByteString.copyFromUtf8(content))) + .setIsExecutable(false) + .setNodeProperties(nodeProperties) + .build(); + } + + public static DirectoryNode makeDirNode(String dirname, Digest dirDigest) { + // Pretty sure we don't need the actual hash for our testing purposes + return DirectoryNode.newBuilder().setName(dirname).setDigest(dirDigest).build(); + } + + public static Digest addDirToTree(Tree.Builder treeBuilder, String dirname, Directory dir) { + ByteString dirnameBytes = ByteString.copyFromUtf8(dirname); + Digest digest = DIGEST_UTIL.compute(dirnameBytes); + String hash = digest.getHash(); + treeBuilder.putDirectories(hash, dir); + return digest; + } + + public static NodeProperties makeNodeProperties(ImmutableMap props) { + return NodeProperties.newBuilder() + .addAllProperties( + props.entrySet().stream() + .map( + kv -> + NodeProperty.newBuilder() + .setName(kv.getKey()) + .setValue(kv.getValue()) + .build()) + .collect(Collectors.toList())) + .build(); + } + + public static Input makeInput(Path fileDir, FileNode file) { + Path fileNodePath = fileDir.resolve(file.getName()); + return Input.newBuilder() + .setPath(fileNodePath.toString()) + .setDigest(file.getDigest().getHashBytes()) + .build(); + } + + public static Command makeCommand() { + ImmutableList outputFiles = ImmutableList.of("output_file", "out_subdir/out_subfile"); + ImmutableList outputDirs = ImmutableList.of("out_subdir"); + ImmutableList outputPaths = + ImmutableList.builder().addAll(outputFiles).addAll(outputDirs).build(); + + return Command.newBuilder() + .addAllOutputFiles(outputFiles) + .addAllOutputDirectories(outputDirs) + .addAllOutputPaths(outputPaths) + .build(); + } + + public static class TreeFile { + public final String path; + public final boolean isTool; + + // null means directory + public final String content; + + public TreeFile(String path) { + this(path, "", false); + } + + public TreeFile(String path, String content) { + this(path, content, false); + } + + public TreeFile(String path, String content, boolean isTool) { + this.path = path; + this.isTool = isTool; + this.content = content; + } + + public boolean isDir() { + return this.content == null; + } + + public String name() { + return Paths.get(this.path).getFileName().toString(); + } + } + + public static Tree makeTree(String rootDirPath, List files) { + Tree.Builder treeBuilder = Tree.newBuilder(); + if (files.isEmpty()) { + return treeBuilder.build(); + } + Directory.Builder rootDirBuilder = Directory.newBuilder(); + + Map dirBuilders = new HashMap<>(); + + for (TreeFile file : files) { + if (file.isDir()) { + dirBuilders.computeIfAbsent(file.path, (filePath) -> Directory.newBuilder()); + } else { + NodeProperties props = NodeProperties.getDefaultInstance(); + if (file.isTool) { + props = makeNodeProperties(ImmutableMap.of(BAZEL_TOOL_INPUT_MARKER, "")); + } + FileNode fileNode = makeFileNode(file.name(), file.content, props); + Path parentDirPath = Paths.get(file.path).getParent(); + if (parentDirPath != null) { + String parentDirPathStr = parentDirPath.normalize().toString(); + Directory.Builder parentDirBuilder = + dirBuilders.computeIfAbsent(parentDirPathStr, (filePath) -> Directory.newBuilder()); + parentDirBuilder.addFiles(fileNode); + } else { + rootDirBuilder.addFiles(fileNode); + } + } + } + + for (Map.Entry entry : dirBuilders.entrySet()) { + String subDirName = entry.getKey(); + Directory subDir = entry.getValue().build(); + Digest subDirDigest = addDirToTree(treeBuilder, subDirName, subDir); + rootDirBuilder.addDirectories(makeDirNode(subDirName, subDirDigest)); + } + + Digest rootDirDigest = addDirToTree(treeBuilder, rootDirPath, rootDirBuilder.build()); + treeBuilder.setRootDigest(rootDirDigest); + + return treeBuilder.build(); + } + + public static List listFilesRec(Path root) throws IOException { + List filesFound = new ArrayList<>(); + + Files.walkFileTree( + root, + new FileVisitor() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) + throws IOException { + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) + throws IOException { + filesFound.add(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException { + throw new IOException("visitFileFailed"); + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + filesFound.add(dir); + return FileVisitResult.CONTINUE; + } + }); + + return filesFound; + } + + // Check all expected files exist and that only they exist + public static void assertFilesExistExactly(Path root, List expectedFiles) + throws IOException { + List listedPaths = listFilesRec(root); + for (Path filePath : listedPaths) { + assertWithMessage("Path not match prefix of any expected file: " + filePath) + .that(expectedFiles.stream().anyMatch(p -> p.startsWith(p))) + .isTrue(); + } + assertThat(listedPaths).containsAtLeastElementsIn(expectedFiles); + } +} From 20cd93e2876543144630c77a6eea7c6c7b8173b2 Mon Sep 17 00:00:00 2001 From: Win Wang Date: Sat, 18 Nov 2023 23:07:58 -0500 Subject: [PATCH 68/68] Address Feedback --- .../java/build/buildfarm/worker/Executor.java | 5 +-- .../build/buildfarm/worker/persistent/BUILD | 2 + .../worker/persistent/FileAccessUtils.java | 23 ++++++++--- .../buildfarm/worker/persistent/Keymaker.java | 14 +++++++ .../worker/persistent/PersistentExecutor.java | 40 +++++++++++++------ .../worker/persistent/ProtoCoordinator.java | 31 +++++++++----- .../worker/persistent/RequestCtx.java | 14 +++++++ .../worker/persistent/ResponseCtx.java | 14 +++++++ .../worker/persistent/WorkFilesContext.java | 14 +++++++ .../worker/persistent/WorkerInputs.java | 25 ++++++++---- .../buildfarm/worker/util/InputsIndexer.java | 14 +++++++ .../persistent/ProtoCoordinatorTest.java | 14 +++++++ .../worker/util/InputsIndexerTest.java | 14 +++++++ .../worker/util/WorkerTestUtils.java | 14 +++++++ 14 files changed, 199 insertions(+), 39 deletions(-) diff --git a/src/main/java/build/buildfarm/worker/Executor.java b/src/main/java/build/buildfarm/worker/Executor.java index 9d89fc428..ee0de0b1b 100644 --- a/src/main/java/build/buildfarm/worker/Executor.java +++ b/src/main/java/build/buildfarm/worker/Executor.java @@ -65,13 +65,10 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.logging.Level; -import java.util.logging.Logger; import lombok.extern.java.Log; @Log class Executor { - private static final Logger logger = Logger.getLogger(Executor.class.getName()); - private static final int INCOMPLETE_EXIT_CODE = -1; private final WorkerContext workerContext; @@ -445,7 +442,7 @@ private Code executeCommand( !limits.persistentWorkerKey.isEmpty() && !limits.persistentWorkerCommand.isEmpty(); if (usePersistentWorker) { - logger.fine( + log.fine( "usePersistentWorker; got persistentWorkerCommand of : " + limits.persistentWorkerCommand); diff --git a/src/main/java/build/buildfarm/worker/persistent/BUILD b/src/main/java/build/buildfarm/worker/persistent/BUILD index 242bc1bfb..b476e0ebb 100644 --- a/src/main/java/build/buildfarm/worker/persistent/BUILD +++ b/src/main/java/build/buildfarm/worker/persistent/BUILD @@ -1,6 +1,7 @@ java_library( name = "persistent", srcs = glob(["*.java"]), + plugins = ["//src/main/java/build/buildfarm/common:lombok"], visibility = ["//visibility:public"], deps = [ "//persistentworkers/src/main/java/persistent/bazel:bazel-persistent-workers", @@ -25,5 +26,6 @@ java_library( "@maven//:io_prometheus_simpleclient", "@maven//:org_apache_commons_commons_compress", "@maven//:org_jetbrains_annotations", + "@maven//:org_projectlombok_lombok", ], ) diff --git a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java index 8410c75dd..ca81384e1 100644 --- a/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java +++ b/src/main/java/build/buildfarm/worker/persistent/FileAccessUtils.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import static java.nio.file.StandardCopyOption.COPY_ATTRIBUTES; @@ -11,18 +25,17 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; -import java.util.logging.Logger; +import lombok.extern.java.Log; /** * Utility for concurrent move/copy of files Can be extended in the future to (sym)linking if we * need performance */ +@Log public final class FileAccessUtils { // singleton class with only static methods private FileAccessUtils() {} - private static final Logger logger = Logger.getLogger(FileAccessUtils.class.getName()); - public static Path addPosixOwnerWrite(Path absPath) throws IOException { Set perms = Files.getPosixFilePermissions(absPath); @@ -54,7 +67,7 @@ private PathLock() {} */ public static void copyFile(Path from, Path to) throws IOException { Path absTo = to.toAbsolutePath(); - logger.finer("copyFile: " + from + " to " + absTo); + log.finer("copyFile: " + from + " to " + absTo); if (!Files.exists(from)) { throw new IOException("copyFile: source file doesn't exist: " + from); } @@ -86,7 +99,7 @@ public static void copyFile(Path from, Path to) throws IOException { */ public static void moveFile(Path from, Path to) throws IOException { Path absTo = to.toAbsolutePath(); - logger.finer("moveFile: " + from + " to " + absTo); + log.finer("moveFile: " + from + " to " + absTo); if (!Files.exists(from)) { throw new IOException("moveFile: source file doesn't exist: " + from); } diff --git a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java index fb6a861ac..edfaaf23c 100644 --- a/src/main/java/build/buildfarm/worker/persistent/Keymaker.java +++ b/src/main/java/build/buildfarm/worker/persistent/Keymaker.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import com.google.common.collect.ImmutableList; diff --git a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java index 5486ff7de..a96d678a0 100644 --- a/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java +++ b/src/main/java/build/buildfarm/worker/persistent/PersistentExecutor.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import build.bazel.remote.execution.v2.ActionResult; @@ -16,8 +30,8 @@ import java.nio.file.Paths; import java.util.Map; import java.util.logging.Level; -import java.util.logging.Logger; import java.util.stream.Collectors; +import lombok.extern.java.Log; import persistent.bazel.client.WorkerKey; /** @@ -27,9 +41,8 @@ * workers, likely for debugging purposes, but need to revisit. (Can't remember fully since it was * so long ago!) */ +@Log public class PersistentExecutor { - private static final Logger logger = Logger.getLogger(PersistentExecutor.class.getName()); - private static final ProtoCoordinator coordinator = ProtoCoordinator.ofCommonsPool(getMaxWorkersPerKey()); @@ -54,7 +67,7 @@ private static int getMaxWorkersPerKey() { try { return Integer.parseInt(System.getenv("BUILDFARM_MAX_WORKERS_PER_KEY")); } catch (Exception ignored) { - logger.info( + log.info( "Could not get env var BUILDFARM_MAX_WORKERS_PER_KEY; defaulting to " + defaultMaxWorkersPerKey); } @@ -79,17 +92,20 @@ public static Code runOnPersistentWorker( throws IOException { //// Pull out persistent worker start command from the overall action request - logger.log(Level.FINE, "executeCommandOnPersistentWorker[" + operationName + "]"); + log.log(Level.FINE, "executeCommandOnPersistentWorker[" + operationName + "]"); ImmutableList initCmd = parseInitCmd(persistentWorkerInitCmd, argsList); String executionName = getExecutionName(argsList); if (executionName.isEmpty()) { - logger.log(Level.SEVERE, "Invalid Argument: " + argsList); + log.log(Level.SEVERE, "Invalid Argument: " + argsList); return Code.INVALID_ARGUMENT; } // TODO revisit why this was necessary in the first place + // (@wiwa) I believe the reason has to do with JavaBuilder workers not relying on env vars, + // as compared to rules_scala, only reading info from the argslist of each command. + // That would mean the Java worker keys should be invariant to the env vars we see. ImmutableMap env; if (executionName.equals(JAVAC_EXEC_NAME)) { env = ImmutableMap.of(); @@ -153,7 +169,7 @@ public static Code runOnPersistentWorker( //// Run request //// Required file operations (in/out) are the responsibility of the coordinator - logger.log(Level.FINE, "Request with key: " + key); + log.log(Level.FINE, "Request with key: " + key); WorkResponse response; String stdErr = ""; try { @@ -171,8 +187,8 @@ public static Code runOnPersistentWorker( + request.getArgumentsList(); String msg = "Exception while running request: " + e + debug + "\n\n"; - logger.log(Level.SEVERE, msg); - e.printStackTrace(); + log.log(Level.SEVERE, msg, e); + response = WorkResponse.newBuilder() .setOutput(msg) @@ -183,7 +199,7 @@ public static Code runOnPersistentWorker( //// Set results String responseOut = response.getOutput(); - logger.log(Level.FINE, "WorkResponse.output: " + responseOut); + log.log(Level.FINE, "WorkResponse.output: " + responseOut); int exitCode = response.getExitCode(); resultBuilder @@ -195,7 +211,7 @@ public static Code runOnPersistentWorker( return Code.OK; } - logger.severe( + log.severe( "PersistentExecutor.runOnPersistentWorker Failed with code: " + exitCode + "\n" @@ -226,7 +242,7 @@ private static ImmutableList parseInitCmd(String cmdStr, ImmutableList initCmdBuilder = ImmutableList.builder(); for (String s : argsList) { - if (cmd.length() == 0) { + if (cmd.isEmpty()) { break; } cmd = cmd.substring(s.length()).trim(); diff --git a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java index 69b02f05f..e3c890225 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java +++ b/src/main/java/build/buildfarm/worker/persistent/ProtoCoordinator.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import static persistent.bazel.client.PersistentWorker.TOOL_INPUT_SUBDIR; @@ -14,7 +28,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.logging.Level; -import java.util.logging.Logger; +import lombok.extern.java.Log; import persistent.bazel.client.CommonsWorkerPool; import persistent.bazel.client.PersistentWorker; import persistent.bazel.client.WorkCoordinator; @@ -26,9 +40,8 @@ * requirements, e.g. ensuring tool input files 3) post-response requirements, i.e. putting output * files in the right place */ +@Log public class ProtoCoordinator extends WorkCoordinator { - private static final Logger logger = Logger.getLogger(ProtoCoordinator.class.getName()); - private static final String WORKER_INIT_LOG_SUFFIX = ".initargs.log"; private static final ConcurrentHashMap pendingReqs = @@ -119,7 +132,7 @@ private static String getUniqueSubdir(Path workRoot) { // copyToolInputsIntoWorkerToolRoot() should have been called before this. private static void copyToolsIntoWorkerExecRoot(WorkerKey key, Path workerExecRoot) throws IOException { - logger.log(Level.FINE, "loadToolsIntoWorkerRoot() into: " + workerExecRoot); + log.log(Level.FINE, "loadToolsIntoWorkerRoot() into: " + workerExecRoot); Path toolInputRoot = key.getExecRoot().resolve(TOOL_INPUT_SUBDIR); for (Path relPath : key.getWorkerFilesWithHashes().keySet()) { @@ -188,9 +201,8 @@ private IOException logBadCleanup(RequestCtx request, IOException e) { .append("getOutputDirectoriesList:\n") .append(context.outputDirectories); - logger.severe(sb.toString()); + log.log(Level.SEVERE, sb.toString(), e); - e.printStackTrace(); return new IOException("Response was OK but failed on postWorkCleanup", e); } @@ -216,11 +228,8 @@ void moveOutputsToOperationRoot(WorkFilesContext context, Path workerExecRoot) } for (String relOutput : context.outputFiles) { - System.out.println(relOutput); Path execOutputPath = workerExecRoot.resolve(relOutput); - System.out.println(execOutputPath); Path opOutputPath = opRoot.resolve(relOutput); - System.out.println(opOutputPath); FileAccessUtils.moveFile(execOutputPath, opOutputPath); } @@ -258,11 +267,11 @@ public void run() { private void onTimeout(RequestCtx request, PersistentWorker worker) { if (worker != null) { - logger.severe("Persistent Worker timed out on request: " + request.request); + log.severe("Persistent Worker timed out on request: " + request.request); try { this.workerPool.invalidateObject(worker.getKey(), worker); } catch (Exception e) { - logger.severe( + log.severe( "Tried to invalidate worker for request:\n" + request + "\n\tbut got: " diff --git a/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java index c051179ca..36f42b2f1 100644 --- a/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java +++ b/src/main/java/build/buildfarm/worker/persistent/RequestCtx.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import com.google.devtools.build.lib.worker.WorkerProtocol.WorkRequest; diff --git a/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java index 0a29b2254..0ff6edcda 100644 --- a/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java +++ b/src/main/java/build/buildfarm/worker/persistent/ResponseCtx.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import com.google.devtools.build.lib.worker.WorkerProtocol.WorkResponse; diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java index 67197172a..4aefc7f29 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkFilesContext.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import build.bazel.remote.execution.v2.Command; diff --git a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java index 4c71a4aa8..de731b9d8 100644 --- a/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java +++ b/src/main/java/build/buildfarm/worker/persistent/WorkerInputs.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import com.google.common.collect.ImmutableMap; @@ -7,11 +21,10 @@ import java.io.IOException; import java.nio.file.Path; import java.util.List; -import java.util.logging.Logger; +import lombok.extern.java.Log; +@Log public class WorkerInputs { - private static final Logger logger = Logger.getLogger(WorkerInputs.class.getName()); - public final Path opRoot; // Some tool inputs are not under opRoot public final ImmutableSet absToolInputs; @@ -38,7 +51,7 @@ public WorkerInputs( for (Path tool : opToolInputs) { if (!allInputs.containsKey(tool)) { String msg = "Tool not found in inputs: " + tool; - logger.severe(msg); + log.severe(msg); throw new IllegalArgumentException(msg); } } @@ -97,9 +110,7 @@ public static WorkerInputs from(WorkFilesContext workFilesContext, List + "\nabsToolInputs: " + absToolInputs; - logger.fine(inputsDebugMsg); - - System.out.println(inputsDebugMsg); + log.fine(inputsDebugMsg); return new WorkerInputs(workFilesContext.opRoot, absToolInputs, toolInputs, pathInputs); } diff --git a/src/main/java/build/buildfarm/worker/util/InputsIndexer.java b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java index 39637db54..84497b04a 100644 --- a/src/main/java/build/buildfarm/worker/util/InputsIndexer.java +++ b/src/main/java/build/buildfarm/worker/util/InputsIndexer.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.util; import build.bazel.remote.execution.v2.Digest; diff --git a/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java b/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java index 998d510cd..908a1746e 100644 --- a/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java +++ b/src/test/java/build/buildfarm/worker/persistent/ProtoCoordinatorTest.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.persistent; import build.bazel.remote.execution.v2.Command; diff --git a/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java index d81048bce..954eb61e4 100644 --- a/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java +++ b/src/test/java/build/buildfarm/worker/util/InputsIndexerTest.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.util; import static build.buildfarm.worker.util.InputsIndexer.BAZEL_TOOL_INPUT_MARKER; diff --git a/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java b/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java index 1a6a687e7..dbaeca5c9 100644 --- a/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java +++ b/src/test/java/build/buildfarm/worker/util/WorkerTestUtils.java @@ -1,3 +1,17 @@ +// Copyright 2023 The Bazel Authors. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package build.buildfarm.worker.util; import static build.buildfarm.worker.util.InputsIndexer.BAZEL_TOOL_INPUT_MARKER;