diff --git a/cmd/bb_runner/BUILD.bazel b/cmd/bb_runner/BUILD.bazel index 28782627..668d7d16 100644 --- a/cmd/bb_runner/BUILD.bazel +++ b/cmd/bb_runner/BUILD.bazel @@ -8,9 +8,9 @@ go_library( importpath = "github.com/buildbarn/bb-remote-execution/cmd/bb_runner", visibility = ["//visibility:private"], deps = [ - "//pkg/environment:go_default_library", "//pkg/proto/configuration/bb_runner:go_default_library", "//pkg/proto/runner:go_default_library", + "//pkg/runner:go_default_library", "@com_github_buildbarn_bb_storage//pkg/filesystem:go_default_library", "@com_github_buildbarn_bb_storage//pkg/global:go_default_library", "@com_github_buildbarn_bb_storage//pkg/grpc:go_default_library", diff --git a/cmd/bb_runner/main.go b/cmd/bb_runner/main.go index 3b166c82..da611b5f 100644 --- a/cmd/bb_runner/main.go +++ b/cmd/bb_runner/main.go @@ -4,9 +4,9 @@ import ( "log" "os" - "github.com/buildbarn/bb-remote-execution/pkg/environment" "github.com/buildbarn/bb-remote-execution/pkg/proto/configuration/bb_runner" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + "github.com/buildbarn/bb-remote-execution/pkg/runner" "github.com/buildbarn/bb-storage/pkg/filesystem" "github.com/buildbarn/bb-storage/pkg/global" bb_grpc "github.com/buildbarn/bb-storage/pkg/grpc" @@ -32,24 +32,25 @@ func main() { log.Fatal("Failed to open build directory: ", err) } - env := environment.NewLocalExecutionEnvironment(buildDirectory, configuration.BuildDirectoryPath) + r := runner.NewLocalRunner( + buildDirectory, + configuration.BuildDirectoryPath) + // When temporary directories need cleaning prior to executing a build - // action, attach a series of TempDirectoryCleaningManagers. - m := environment.NewSingletonManager(env) + // action, attach a series of TemporaryDirectoryCleaningRunners. for _, d := range configuration.TemporaryDirectories { directory, err := filesystem.NewLocalDirectory(d) if err != nil { log.Fatalf("Failed to open temporary directory %#v: %s", d, err) } - m = environment.NewTempDirectoryCleaningManager(m, directory) + r = runner.NewTemporaryDirectoryCleaningRunner(r, directory, d) } - runnerServer := environment.NewRunnerServer(environment.NewConcurrentManager(m)) log.Fatal( "gRPC server failure: ", bb_grpc.NewGRPCServersFromConfigurationAndServe( configuration.GrpcServers, func(s *grpc.Server) { - runner.RegisterRunnerServer(s, runnerServer) + runner_pb.RegisterRunnerServer(s, runner.NewRunnerServer(r)) })) } diff --git a/cmd/bb_worker/BUILD.bazel b/cmd/bb_worker/BUILD.bazel index 5b2e0317..fe7a39a6 100644 --- a/cmd/bb_worker/BUILD.bazel +++ b/cmd/bb_worker/BUILD.bazel @@ -11,11 +11,12 @@ go_library( "//pkg/blobstore:go_default_library", "//pkg/builder:go_default_library", "//pkg/cas:go_default_library", - "//pkg/environment:go_default_library", "//pkg/filesystem:go_default_library", "//pkg/proto/configuration/bb_worker:go_default_library", "//pkg/proto/remoteworker:go_default_library", "//pkg/proto/runner:go_default_library", + "//pkg/runner:go_default_library", + "//pkg/sync:go_default_library", "@com_github_buildbarn_bb_storage//pkg/blobstore:go_default_library", "@com_github_buildbarn_bb_storage//pkg/blobstore/configuration:go_default_library", "@com_github_buildbarn_bb_storage//pkg/cas:go_default_library", diff --git a/cmd/bb_worker/main.go b/cmd/bb_worker/main.go index 11b787de..6e4878f7 100644 --- a/cmd/bb_worker/main.go +++ b/cmd/bb_worker/main.go @@ -15,11 +15,12 @@ import ( re_blobstore "github.com/buildbarn/bb-remote-execution/pkg/blobstore" "github.com/buildbarn/bb-remote-execution/pkg/builder" re_cas "github.com/buildbarn/bb-remote-execution/pkg/cas" - "github.com/buildbarn/bb-remote-execution/pkg/environment" re_filesystem "github.com/buildbarn/bb-remote-execution/pkg/filesystem" "github.com/buildbarn/bb-remote-execution/pkg/proto/configuration/bb_worker" "github.com/buildbarn/bb-remote-execution/pkg/proto/remoteworker" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + "github.com/buildbarn/bb-remote-execution/pkg/runner" + "github.com/buildbarn/bb-remote-execution/pkg/sync" "github.com/buildbarn/bb-storage/pkg/blobstore" blobstore_configuration "github.com/buildbarn/bb-storage/pkg/blobstore/configuration" "github.com/buildbarn/bb-storage/pkg/cas" @@ -82,7 +83,7 @@ func main() { log.Fatal("Failed to create blob access: ", err) } - var buildDirectory filesystem.Directory + var naiveBuildDirectory filesystem.DirectoryCloser switch buildDirectoryConfigurationVariant := configuration.BuildDirectory.(type) { case *bb_worker.ApplicationConfiguration_LocalBuildDirectory: // To ease privilege separation, clear the umask. This @@ -93,16 +94,10 @@ func main() { // Directory where actual builds take place. buildDirectoryConfiguration := buildDirectoryConfigurationVariant.LocalBuildDirectory - buildDirectory, err = filesystem.NewLocalDirectory(buildDirectoryConfiguration.BuildDirectoryPath) + naiveBuildDirectory, err = filesystem.NewLocalDirectory(buildDirectoryConfiguration.BuildDirectoryPath) if err != nil { log.Fatal("Failed to open build directory: ", err) } - // TODO: This may be removed when the - // CleanBuildDirectoryManager is enabled unconditionally - // once again. - if err := buildDirectory.RemoveAllChildren(); err != nil { - log.Fatal("Failed to clean build directory on startup: ", err) - } default: log.Fatal("No build directory specified") } @@ -143,6 +138,8 @@ func main() { eviction.NewMetricsSet(evictionSet, "HardlinkingContentAddressableStorage")) } + var buildDirectoryInitializer sync.Initializer + var sharedBuildDirectoryNextParallelActionID uint64 if len(configuration.Runners) == 0 { log.Fatal("Cannot start worker without any runners") } @@ -173,7 +170,7 @@ func main() { } // Wait for the runner process to come online. - runnerClient := runner.NewRunnerClient(runnerConnection) + runnerClient := runner_pb.NewRunnerClient(runnerConnection) for { _, err := runnerClient.CheckReadiness(context.Background(), &empty.Empty{}) if err == nil { @@ -183,32 +180,6 @@ func main() { time.Sleep(3 * time.Second) } - // Build environment capable of executing one action at a time. - // The build takes place in the root of the build directory. - environmentManager := environment.NewSingletonManager( - environment.NewRemoteExecutionEnvironment(runnerConnection, buildDirectory)) - - // Clean the build directory every time when going from - // fully idle to executing one action. - // TODO: Also enable this feature when multiple runners - // are configured. Being able to support this requires a - // decomposition of environment.Environment into two - // separate interfaces: one for managing the build - // directory and one for running commands. - if len(configuration.Runners) == 1 { - environmentManager = environment.NewCleanBuildDirectoryManager(environmentManager) - } - - // Create a per-action subdirectory in the build directory named - // after the action digest, so that multiple actions may be run - // concurrently within the same environment. - // TODO(edsch): It might make sense to disable this if - // concurrency is disabled to improve action cache hit rate, but - // only if there are no other workers in the same cluster that - // have concurrency enabled. - environmentManager = environment.NewActionDigestSubdirectoryManager( - environment.NewConcurrentManager(environmentManager)) - for threadID := uint64(0); threadID < runnerConfiguration.Concurrency; threadID++ { go func(runnerConfiguration *bb_worker.RunnerConfiguration, threadID uint64) { // Per-worker separate writer of the Content @@ -227,13 +198,28 @@ func main() { contentAddressableStorageWriter, int(configuration.MaximumMessageSizeBytes))) - var inputRootPopulator builder.InputRootPopulator + var buildDirectory builder.BuildDirectory switch configuration.BuildDirectory.(type) { case *bb_worker.ApplicationConfiguration_LocalBuildDirectory: - inputRootPopulator = builder.NewNaiveInputRootPopulator( + buildDirectory = builder.NewNaiveBuildDirectory( + naiveBuildDirectory, contentAddressableStorage) } + // Create a per-action subdirectory in + // the build directory named after the + // action digest, so that multiple + // actions may be run concurrently. + // + // Also clean the build directory every + // time when going from fully idle to + // executing one action. + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator( + builder.NewCleanBuildDirectoryCreator( + builder.NewRootBuildDirectoryCreator(buildDirectory), + &buildDirectoryInitializer), + &sharedBuildDirectoryNextParallelActionID) + workerID := map[string]string{} if runnerConfiguration.Concurrency > 1 { workerID["thread"] = fmt.Sprintf("%0*d", concurrencyLength, threadID) @@ -254,8 +240,8 @@ func main() { builder.NewStorageFlushingBuildExecutor( builder.NewLocalBuildExecutor( contentAddressableStorage, - environmentManager, - inputRootPopulator, + buildDirectoryCreator, + runner.NewRemoteRunner(runnerConnection), clock.SystemClock, defaultExecutionTimeout, maximumExecutionTimeout), diff --git a/internal/mock/BUILD.bazel b/internal/mock/BUILD.bazel index 41be247e..b0e7e6b6 100644 --- a/internal/mock/BUILD.bazel +++ b/internal/mock/BUILD.bazel @@ -15,8 +15,9 @@ gomock( name = "builder", out = "builder.go", interfaces = [ + "BuildDirectory", + "BuildDirectoryCreator", "BuildExecutor", - "InputRootPopulator", "StorageFlusher", ], library = "//pkg/builder:go_default_library", @@ -42,18 +43,6 @@ gomock( package = "mock", ) -gomock( - name = "environment", - out = "environment.go", - interfaces = [ - "Environment", - "ManagedEnvironment", - "Manager", - ], - library = "//pkg/environment:go_default_library", - package = "mock", -) - gomock( name = "filesystem", out = "filesystem.go", @@ -87,6 +76,14 @@ gomock( package = "mock", ) +gomock( + name = "runner", + out = "runner.go", + interfaces = ["Runner"], + library = "//pkg/runner:go_default_library", + package = "mock", +) + gomock( name = "sharding", out = "sharding.go", @@ -132,10 +129,10 @@ go_library( ":builder.go", ":cas.go", ":clock.go", - ":environment.go", ":filesystem.go", ":filesystem_re.go", ":remoteexecution.go", + ":runner.go", ":sharding.go", ":storage_builder.go", ":storage_util.go", @@ -145,7 +142,6 @@ go_library( visibility = ["//:__subpackages__"], deps = [ "//pkg/builder:go_default_library", - "//pkg/environment:go_default_library", "//pkg/filesystem:go_default_library", "//pkg/proto/remoteworker:go_default_library", "//pkg/proto/runner:go_default_library", diff --git a/pkg/builder/BUILD.bazel b/pkg/builder/BUILD.bazel index 516821e8..99c1dc9b 100644 --- a/pkg/builder/BUILD.bazel +++ b/pkg/builder/BUILD.bazel @@ -4,27 +4,32 @@ go_library( name = "go_default_library", srcs = [ "build_client.go", + "build_directory.go", + "build_directory_creator.go", "build_executor.go", "build_queue_state_provider.go", "caching_build_executor.go", + "clean_build_directory_creator.go", "file_pool_stats_build_executor.go", "in_memory_build_queue.go", - "input_root_populator.go", "local_build_executor.go", "logging_build_executor.go", "metrics_build_executor.go", - "naive_input_root_populator.go", + "naive_build_directory.go", + "root_build_directory_creator.go", + "shared_build_directory_creator.go", "storage_flushing_build_executor.go", "timestamped_build_executor.go", ], importpath = "github.com/buildbarn/bb-remote-execution/pkg/builder", visibility = ["//visibility:public"], deps = [ - "//pkg/environment:go_default_library", "//pkg/filesystem:go_default_library", "//pkg/proto/remoteworker:go_default_library", "//pkg/proto/resourceusage:go_default_library", "//pkg/proto/runner:go_default_library", + "//pkg/runner:go_default_library", + "//pkg/sync:go_default_library", "//pkg/util:go_default_library", "@com_github_bazelbuild_remote_apis//build/bazel/remote/execution/v2:go_default_library", "@com_github_bazelbuild_remote_apis//build/bazel/semver:go_default_library", @@ -55,10 +60,13 @@ go_test( name = "go_default_test", srcs = [ "caching_build_executor_test.go", + "clean_build_directory_creator_test.go", "file_pool_stats_build_executor_test.go", "in_memory_build_queue_test.go", "local_build_executor_test.go", - "naive_input_root_populator_test.go", + "naive_build_directory_test.go", + "root_build_directory_creator_test.go", + "shared_build_directory_creator_test.go", "storage_flushing_build_executor_test.go", "timestamped_build_executor_test.go", ], @@ -69,6 +77,7 @@ go_test( "//pkg/proto/remoteworker:go_default_library", "//pkg/proto/resourceusage:go_default_library", "//pkg/proto/runner:go_default_library", + "//pkg/sync:go_default_library", "@com_github_bazelbuild_remote_apis//build/bazel/remote/execution/v2:go_default_library", "@com_github_buildbarn_bb_storage//pkg/blobstore/buffer:go_default_library", "@com_github_buildbarn_bb_storage//pkg/builder:go_default_library", diff --git a/pkg/builder/build_directory.go b/pkg/builder/build_directory.go new file mode 100644 index 00000000..2ebf9309 --- /dev/null +++ b/pkg/builder/build_directory.go @@ -0,0 +1,32 @@ +package builder + +import ( + "context" + + re_filesystem "github.com/buildbarn/bb-remote-execution/pkg/filesystem" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/filesystem" +) + +// BuildDirectory is a filesystem.Directory that may be used for the +// purpose of running build actions. BuildDirectory has a couple of +// special operations that implementations may use to run actions in a +// more efficient and manageable way. +type BuildDirectory interface { + filesystem.DirectoryCloser + + // Identical to EnterDirectory(), except that it returns a + // BuildDirectory object. + EnterBuildDirectory(name string) (BuildDirectory, error) + + // Installs a set of hooks into the directory to intercept I/O + // operations. The FilePool may be used to allocate storage + // space. Implementations of BuildDirectory are free to let this + // be a no-op, with the disadvantage that they cannot apply + // resource limits or provide rich I/O error messages. + InstallHooks(filePool re_filesystem.FilePool) + + // Recursively merges the contents of a Directory stored in the + // Content Addressable Storage into a local directory. + MergeDirectoryContents(ctx context.Context, digest digest.Digest) error +} diff --git a/pkg/builder/build_directory_creator.go b/pkg/builder/build_directory_creator.go new file mode 100644 index 00000000..1ff8cfc3 --- /dev/null +++ b/pkg/builder/build_directory_creator.go @@ -0,0 +1,11 @@ +package builder + +import ( + "github.com/buildbarn/bb-storage/pkg/digest" +) + +// BuildDirectoryCreator is used by LocalBuildExecutor to obtain build +// directories in which build actions are executed. +type BuildDirectoryCreator interface { + GetBuildDirectory(actionDigest digest.Digest, mayRunInParallel bool) (BuildDirectory, string, error) +} diff --git a/pkg/builder/clean_build_directory_creator.go b/pkg/builder/clean_build_directory_creator.go new file mode 100644 index 00000000..8f5b71d9 --- /dev/null +++ b/pkg/builder/clean_build_directory_creator.go @@ -0,0 +1,50 @@ +package builder + +import ( + "github.com/buildbarn/bb-remote-execution/pkg/sync" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/util" + + "google.golang.org/grpc/codes" +) + +type cleanBuildDirectoryCreator struct { + base BuildDirectoryCreator + initializer *sync.Initializer +} + +// NewCleanBuildDirectoryCreator is an adapter for BuildDirectoryCreator +// that upon acquistion empties out the build directory. This ensures +// that the build action is executed in a clean environment. +func NewCleanBuildDirectoryCreator(base BuildDirectoryCreator, initializer *sync.Initializer) BuildDirectoryCreator { + return &cleanBuildDirectoryCreator{ + base: base, + initializer: initializer, + } +} + +func (dc *cleanBuildDirectoryCreator) GetBuildDirectory(actionDigest digest.Digest, mayRunInParallel bool) (BuildDirectory, string, error) { + buildDirectory, buildDirectoryPath, err := dc.base.GetBuildDirectory(actionDigest, mayRunInParallel) + if err != nil { + return nil, "", err + } + if err := dc.initializer.Acquire(buildDirectory.RemoveAllChildren); err != nil { + buildDirectory.Close() + return nil, "", util.StatusWrapfWithCode(err, codes.Internal, "Failed to clean build directory %#v prior to build", buildDirectoryPath) + } + return &cleanBuildDirectory{ + BuildDirectory: buildDirectory, + initializer: dc.initializer, + }, buildDirectoryPath, nil +} + +type cleanBuildDirectory struct { + BuildDirectory + initializer *sync.Initializer +} + +func (d cleanBuildDirectory) Close() error { + err := d.BuildDirectory.Close() + d.initializer.Release() + return err +} diff --git a/pkg/builder/clean_build_directory_creator_test.go b/pkg/builder/clean_build_directory_creator_test.go new file mode 100644 index 00000000..e1c7ee05 --- /dev/null +++ b/pkg/builder/clean_build_directory_creator_test.go @@ -0,0 +1,84 @@ +package builder_test + +import ( + "os" + "testing" + + "github.com/buildbarn/bb-remote-execution/internal/mock" + "github.com/buildbarn/bb-remote-execution/pkg/builder" + "github.com/buildbarn/bb-remote-execution/pkg/sync" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +func TestCleanBuildDirectoryCreatorAcquireFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Failure to create a build directory should simply be forwarded. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(nil, "", status.Error(codes.Internal, "No space left on device")) + + var initializer sync.Initializer + buildDirectoryCreator := builder.NewCleanBuildDirectoryCreator(baseBuildDirectoryCreator, &initializer) + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.Equal(t, status.Error(codes.Internal, "No space left on device"), err) +} + +func TestCleanBuildDirectoryCreatorRemoveAllChildrenFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Failure to clean the build subdirectory is always an internal error. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().RemoveAllChildren().Return( + status.Error(codes.PermissionDenied, "You don't have permissions to remove files from disk")) + baseBuildDirectory.EXPECT().Close() + + var initializer sync.Initializer + buildDirectoryCreator := builder.NewCleanBuildDirectoryCreator(baseBuildDirectoryCreator, &initializer) + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.Equal(t, status.Error(codes.Internal, "Failed to clean build directory \"base-directory\" prior to build: You don't have permissions to remove files from disk"), err) +} + +func TestCleanBuildDirectoryCreatorSuccess(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Successful build in a clean build directory. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().RemoveAllChildren().Return(nil) + baseBuildDirectory.EXPECT().Mkdir("hello", os.FileMode(0700)) + baseBuildDirectory.EXPECT().Close() + + var initializer sync.Initializer + buildDirectoryCreator := builder.NewCleanBuildDirectoryCreator(baseBuildDirectoryCreator, &initializer) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.NoError(t, err) + require.Equal(t, "base-directory", buildDirectoryPath) + require.NoError(t, buildDirectory.Mkdir("hello", os.FileMode(0700))) + buildDirectory.Close() +} diff --git a/pkg/builder/input_root_populator.go b/pkg/builder/input_root_populator.go deleted file mode 100644 index 69e0af43..00000000 --- a/pkg/builder/input_root_populator.go +++ /dev/null @@ -1,18 +0,0 @@ -package builder - -import ( - "context" - - re_filesystem "github.com/buildbarn/bb-remote-execution/pkg/filesystem" - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/buildbarn/bb-storage/pkg/filesystem" -) - -// InputRootPopulator implements the strategy for populating the build -// directory with the desired contents, so that it may be built. The -// simplest way of populating the input root is to recursively create -// all files and directories. A more complex strategy may use -// parallelism. -type InputRootPopulator interface { - PopulateInputRoot(ctx context.Context, filePool re_filesystem.FilePool, digest digest.Digest, inputRoot filesystem.Directory) error -} diff --git a/pkg/builder/local_build_executor.go b/pkg/builder/local_build_executor.go index a0d5501c..1f2cafd6 100644 --- a/pkg/builder/local_build_executor.go +++ b/pkg/builder/local_build_executor.go @@ -8,10 +8,10 @@ import ( "time" remoteexecution "github.com/bazelbuild/remote-apis/build/bazel/remote/execution/v2" - "github.com/buildbarn/bb-remote-execution/pkg/environment" re_filesystem "github.com/buildbarn/bb-remote-execution/pkg/filesystem" "github.com/buildbarn/bb-remote-execution/pkg/proto/remoteworker" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + "github.com/buildbarn/bb-remote-execution/pkg/runner" "github.com/buildbarn/bb-storage/pkg/cas" "github.com/buildbarn/bb-storage/pkg/clock" "github.com/buildbarn/bb-storage/pkg/digest" @@ -27,8 +27,8 @@ import ( type localBuildExecutor struct { contentAddressableStorage cas.ContentAddressableStorage - environmentManager environment.Manager - inputRootPopulator InputRootPopulator + buildDirectoryCreator BuildDirectoryCreator + runner runner.Runner clock clock.Clock defaultExecutionTimeout time.Duration maximumExecutionTimeout time.Duration @@ -36,11 +36,11 @@ type localBuildExecutor struct { // NewLocalBuildExecutor returns a BuildExecutor that executes build // steps on the local system. -func NewLocalBuildExecutor(contentAddressableStorage cas.ContentAddressableStorage, environmentManager environment.Manager, inputRootPopulator InputRootPopulator, clock clock.Clock, defaultExecutionTimeout time.Duration, maximumExecutionTimeout time.Duration) BuildExecutor { +func NewLocalBuildExecutor(contentAddressableStorage cas.ContentAddressableStorage, buildDirectoryCreator BuildDirectoryCreator, runner runner.Runner, clock clock.Clock, defaultExecutionTimeout time.Duration, maximumExecutionTimeout time.Duration) BuildExecutor { return &localBuildExecutor{ contentAddressableStorage: contentAddressableStorage, - environmentManager: environmentManager, - inputRootPopulator: inputRootPopulator, + buildDirectoryCreator: buildDirectoryCreator, + runner: runner, clock: clock, defaultExecutionTimeout: defaultExecutionTimeout, maximumExecutionTimeout: maximumExecutionTimeout, @@ -131,10 +131,10 @@ func (be *localBuildExecutor) uploadTree(ctx context.Context, outputDirectory fi return treeDigest, err } -func (be *localBuildExecutor) createOutputParentDirectory(buildDirectory filesystem.Directory, outputParentPath string) (filesystem.DirectoryCloser, error) { +func (be *localBuildExecutor) createOutputParentDirectory(inputRootDirectory filesystem.Directory, outputParentPath string) (filesystem.DirectoryCloser, error) { // Create and enter successive components, closing the former. components := strings.FieldsFunc(outputParentPath, func(r rune) bool { return r == '/' }) - d := filesystem.NopDirectoryCloser(buildDirectory) + d := filesystem.NopDirectoryCloser(inputRootDirectory) for n, component := range components { if component != "." { if err := d.Mkdir(component, 0777); err != nil && !os.IsExist(err) { @@ -189,7 +189,7 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste } } - // Obtain build environment. + // Obtain build directory. actionDigest, err := digest.NewDigestFromPartialDigest(instanceName, request.ActionDigest) if err != nil { attachErrorToExecuteResponse(response, util.StatusWrap(err, "Failed to extract digest for action")) @@ -200,14 +200,18 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste attachErrorToExecuteResponse(response, status.Error(codes.InvalidArgument, "Request does not contain a command")) return response } - environment, err := be.environmentManager.Acquire(actionDigest) + buildDirectory, buildDirectoryPath, err := be.buildDirectoryCreator.GetBuildDirectory(actionDigest, action.DoNotCache) if err != nil { attachErrorToExecuteResponse( response, util.StatusWrap(err, "Failed to acquire build environment")) return response } - defer environment.Release() + defer buildDirectory.Close() + + // Install hooks on build directory to capture file creation + // events. + buildDirectory.InstallHooks(filePool) executionStateUpdates <- &remoteworker.CurrentState_Executing{ ActionDigest: request.ActionDigest, @@ -216,7 +220,22 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste }, } - // Set up inputs. + // Create input root directory inside of build directory. + if err := buildDirectory.Mkdir("root", 0777); err != nil { + attachErrorToExecuteResponse( + response, + util.StatusWrap(err, "Failed to create input root directory")) + return response + } + inputRootDirectory, err := buildDirectory.EnterBuildDirectory("root") + if err != nil { + attachErrorToExecuteResponse( + response, + util.StatusWrap(err, "Failed to enter input root directory")) + return response + } + defer inputRootDirectory.Close() + inputRootDigest, err := actionDigest.NewDerivedDigest(action.InputRootDigest) if err != nil { attachErrorToExecuteResponse( @@ -224,10 +243,7 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste util.StatusWrap(err, "Failed to extract digest for input root")) return response } - buildDirectory := environment.GetBuildDirectory() - ctxWithIOError, cancelIOError := context.WithCancel(ctx) - defer cancelIOError() - if err := be.inputRootPopulator.PopulateInputRoot(ctx, filePool, inputRootDigest, buildDirectory); err != nil { + if err := inputRootDirectory.MergeDirectoryContents(ctx, inputRootDigest); err != nil { attachErrorToExecuteResponse(response, err) return response } @@ -253,29 +269,25 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste // possibility of omitting output directories and using // OutputDirectorySymlinks. if _, ok := outputParentDirectories[outputDirectory]; !ok { - dir, err := be.createOutputParentDirectory(buildDirectory, path.Join(command.WorkingDirectory, outputDirectory)) + dir, err := be.createOutputParentDirectory(inputRootDirectory, path.Join(command.WorkingDirectory, outputDirectory)) if err != nil { attachErrorToExecuteResponse(response, err) return response } outputParentDirectories[outputDirectory] = dir - if dir != buildDirectory { - defer dir.Close() - } + defer dir.Close() } } for _, outputFile := range command.OutputFiles { dirPath := path.Dir(outputFile) if _, ok := outputParentDirectories[dirPath]; !ok { - dir, err := be.createOutputParentDirectory(buildDirectory, path.Join(command.WorkingDirectory, dirPath)) + dir, err := be.createOutputParentDirectory(inputRootDirectory, path.Join(command.WorkingDirectory, dirPath)) if err != nil { attachErrorToExecuteResponse(response, err) return response } outputParentDirectories[dirPath] = dir - if dir != buildDirectory { - defer dir.Close() - } + defer dir.Close() } } @@ -287,18 +299,19 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste } // Invoke the command. - ctxWithTimeout, cancelTimeout := be.clock.NewContextWithTimeout(ctxWithIOError, executionTimeout) + ctxWithTimeout, cancelTimeout := be.clock.NewContextWithTimeout(ctx, executionTimeout) defer cancelTimeout() environmentVariables := map[string]string{} for _, environmentVariable := range command.EnvironmentVariables { environmentVariables[environmentVariable.Name] = environmentVariable.Value } - runResponse, runErr := environment.Run(ctxWithTimeout, &runner.RunRequest{ + runResponse, runErr := be.runner.Run(ctxWithTimeout, &runner_pb.RunRequest{ Arguments: command.Arguments, EnvironmentVariables: environmentVariables, WorkingDirectory: command.WorkingDirectory, - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", + StdoutPath: path.Join(buildDirectoryPath, "stdout"), + StderrPath: path.Join(buildDirectoryPath, "stderr"), + InputRootDirectory: path.Join(buildDirectoryPath, "root"), }) // Attach the exit code or execution error. @@ -319,12 +332,12 @@ func (be *localBuildExecutor) Execute(ctx context.Context, filePool re_filesyste // Upload command output. In the common case, the files are // empty. If that's the case, don't bother setting the digest to // keep the ActionResult small. - if stdoutDigest, err := be.contentAddressableStorage.PutFile(ctx, buildDirectory, ".stdout.txt", actionDigest); err != nil { + if stdoutDigest, err := be.contentAddressableStorage.PutFile(ctx, buildDirectory, "stdout", actionDigest); err != nil { attachErrorToExecuteResponse(response, util.StatusWrap(err, "Failed to store stdout")) } else if stdoutDigest.GetSizeBytes() > 0 { response.Result.StdoutDigest = stdoutDigest.GetPartialDigest() } - if stderrDigest, err := be.contentAddressableStorage.PutFile(ctx, buildDirectory, ".stderr.txt", actionDigest); err != nil { + if stderrDigest, err := be.contentAddressableStorage.PutFile(ctx, buildDirectory, "stderr", actionDigest); err != nil { attachErrorToExecuteResponse(response, util.StatusWrap(err, "Failed to store stderr")) } else if stderrDigest.GetSizeBytes() > 0 { response.Result.StderrDigest = stderrDigest.GetPartialDigest() diff --git a/pkg/builder/local_build_executor_test.go b/pkg/builder/local_build_executor_test.go index 62e32954..5129355b 100644 --- a/pkg/builder/local_build_executor_test.go +++ b/pkg/builder/local_build_executor_test.go @@ -11,7 +11,7 @@ import ( "github.com/buildbarn/bb-remote-execution/internal/mock" "github.com/buildbarn/bb-remote-execution/pkg/builder" "github.com/buildbarn/bb-remote-execution/pkg/proto/remoteworker" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" "github.com/buildbarn/bb-storage/pkg/digest" "github.com/buildbarn/bb-storage/pkg/filesystem" "github.com/buildbarn/bb-storage/pkg/util" @@ -26,21 +26,14 @@ import ( "google.golang.org/grpc/status" ) -func mustStatus(s *status.Status, err error) *status.Status { - if err != nil { - panic("Failed to create status") - } - return s -} - -func TestLocalBuildExecutorEnvironmentInvalidActionDigest(t *testing.T) { +func TestLocalBuildExecutorInvalidActionDigest(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) filePool := mock.NewMockFilePool(ctrl) metadata := make(chan *remoteworker.CurrentState_Executing, 10) @@ -76,14 +69,14 @@ func TestLocalBuildExecutorEnvironmentInvalidActionDigest(t *testing.T) { }, executeResponse) } -func TestLocalBuildExecutorEnvironmentMissingAction(t *testing.T) { +func TestLocalBuildExecutorMissingAction(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) filePool := mock.NewMockFilePool(ctrl) metadata := make(chan *remoteworker.CurrentState_Executing, 10) @@ -113,14 +106,14 @@ func TestLocalBuildExecutorEnvironmentMissingAction(t *testing.T) { }, executeResponse) } -func TestLocalBuildExecutorEnvironmentMissingCommand(t *testing.T) { +func TestLocalBuildExecutorMissingCommand(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) filePool := mock.NewMockFilePool(ctrl) metadata := make(chan *remoteworker.CurrentState_Executing, 10) @@ -149,17 +142,18 @@ func TestLocalBuildExecutorEnvironmentMissingCommand(t *testing.T) { }, executeResponse) } -func TestLocalBuildExecutorEnvironmentAcquireFailed(t *testing.T) { +func TestLocalBuildExecutorBuildDirectoryCreatorFailedFailed(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("netbsd", "5555555555555555555555555555555555555555555555555555555555555555", 7), - ).Return(nil, status.Error(codes.InvalidArgument, "Platform requirements not provided")) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + false, + ).Return(nil, "", status.Error(codes.InvalidArgument, "Platform requirements not provided")) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) filePool := mock.NewMockFilePool(ctrl) metadata := make(chan *remoteworker.CurrentState_Executing, 10) @@ -199,23 +193,26 @@ func TestLocalBuildExecutorInputRootPopulationFailed(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("netbsd", "5555555555555555555555555555555555555555555555555555555555555555", 7), - ).Return(environment, nil) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) - buildDirectory := mock.NewMockDirectory(ctrl) - environment.EXPECT().GetBuildDirectory().Return(buildDirectory) + false, + ).Return(buildDirectory, ".", nil) filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().InstallHooks(filePool) + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory).Return(status.Error(codes.FailedPrecondition, "Some input files could not be found")) - environment.EXPECT().Release() + ).Return(status.Error(codes.FailedPrecondition, "Some input files could not be found")) + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -254,24 +251,27 @@ func TestLocalBuildExecutorOutputDirectoryCreationFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("fedora", "5555555555555555555555555555555555555555555555555555555555555555", 7), - ).Return(environment, nil) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) - buildDirectory := mock.NewMockDirectory(ctrl) - environment.EXPECT().GetBuildDirectory().Return(buildDirectory) + false, + ).Return(buildDirectory, ".", nil) filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().InstallHooks(filePool) + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("fedora", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory).Return(nil) - buildDirectory.EXPECT().Mkdir("foo", os.FileMode(0777)).Return(status.Error(codes.Internal, "Out of disk space")) - environment.EXPECT().Release() + ).Return(nil) + inputRootDirectory.EXPECT().Mkdir("foo", os.FileMode(0777)).Return(status.Error(codes.Internal, "Out of disk space")) + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -310,49 +310,53 @@ func TestLocalBuildExecutorOutputSymlinkReadingFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - buildDirectory := mock.NewMockDirectory(ctrl) - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stdout.txt", gomock.Any()).Return( + buildDirectory := mock.NewMockBuildDirectory(ctrl) + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stdout", gomock.Any()).Return( digest.MustNewDigest("nintendo64", "0000000000000000000000000000000000000000000000000000000000000005", 567), nil) - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stderr.txt", gomock.Any()).Return( + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stderr", gomock.Any()).Return( digest.MustNewDigest("nintendo64", "0000000000000000000000000000000000000000000000000000000000000006", 678), nil) - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("nintendo64", "5555555555555555555555555555555555555555555555555555555555555555", 7), - ).Return(environment, nil) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) - environment.EXPECT().GetBuildDirectory().Return(buildDirectory) + false, + ).Return(buildDirectory, ".", nil) filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().InstallHooks(filePool) + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("nintendo64", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory).Return(nil) - buildDirectory.EXPECT().Mkdir("foo", os.FileMode(0777)).Return(nil) - environment.EXPECT().Run(gomock.Any(), &runner.RunRequest{ + ).Return(nil) + inputRootDirectory.EXPECT().Mkdir("foo", os.FileMode(0777)).Return(nil) + runner := mock.NewMockRunner(ctrl) + runner.EXPECT().Run(gomock.Any(), &runner_pb.RunRequest{ Arguments: []string{"touch", "foo"}, EnvironmentVariables: map[string]string{"PATH": "/bin:/usr/bin"}, WorkingDirectory: "", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }).Return(&runner.RunResponse{ + StdoutPath: "stdout", + StderrPath: "stderr", + InputRootDirectory: "root", + }).Return(&runner_pb.RunResponse{ ExitCode: 0, }, nil) - environment.EXPECT().Release() fooDirectory := mock.NewMockDirectoryCloser(ctrl) - buildDirectory.EXPECT().EnterDirectory("foo").Return(fooDirectory, nil) + inputRootDirectory.EXPECT().EnterDirectory("foo").Return(fooDirectory, nil) fooDirectory.EXPECT().ReadDir().Return([]filesystem.FileInfo{ filesystem.NewFileInfo("bar", filesystem.FileTypeSymlink), }, nil) fooDirectory.EXPECT().Readlink("bar").Return("", status.Error(codes.Internal, "Cosmic rays caused interference")) fooDirectory.EXPECT().Close() + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() clock := mock.NewMockClock(ctrl) clock.EXPECT().NewContextWithTimeout(gomock.Any(), time.Hour).DoAndReturn(func(parent context.Context, timeout time.Duration) (context.Context, context.CancelFunc) { return context.WithCancel(parent) }) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -401,15 +405,16 @@ func TestLocalBuildExecutorSuccess(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() - // File system operations that should occur against the build directory. - // Creation of bazel-out/k8-fastbuild/bin/_objs/hello. - buildDirectory := mock.NewMockDirectory(ctrl) - buildDirectory.EXPECT().Mkdir("bazel-out", os.FileMode(0777)).Return(nil) + // File system operations that should occur against the input + // root directory. Creation of + // bazel-out/k8-fastbuild/bin/_objs/hello. + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + inputRootDirectory.EXPECT().Mkdir("bazel-out", os.FileMode(0777)).Return(nil) bazelOutDirectory := mock.NewMockDirectoryCloser(ctrl) - buildDirectory.EXPECT().EnterDirectory("bazel-out").Return(bazelOutDirectory, nil) + inputRootDirectory.EXPECT().EnterDirectory("bazel-out").Return(bazelOutDirectory, nil) bazelOutDirectory.EXPECT().Close() bazelOutDirectory.EXPECT().Mkdir("k8-fastbuild", os.FileMode(0777)).Return(nil) - k8FastbuildDirectory := mock.NewMockDirectoryCloser(ctrl) + k8FastbuildDirectory := mock.NewMockBuildDirectory(ctrl) bazelOutDirectory.EXPECT().EnterDirectory("k8-fastbuild").Return(k8FastbuildDirectory, nil) k8FastbuildDirectory.EXPECT().Close() k8FastbuildDirectory.EXPECT().Mkdir("bin", os.FileMode(0777)).Return(nil) @@ -431,10 +436,11 @@ func TestLocalBuildExecutorSuccess(t *testing.T) { contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) // Write operations against the Content Addressable Storage. - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stdout.txt", gomock.Any()).Return( + buildDirectory := mock.NewMockBuildDirectory(ctrl) + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stdout", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000005", 567), nil) - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stderr.txt", gomock.Any()).Return( + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stderr", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000006", 678), nil) contentAddressableStorage.EXPECT().PutFile(ctx, helloDirectory, "hello.pic.d", gomock.Any()).Return( @@ -445,22 +451,23 @@ func TestLocalBuildExecutorSuccess(t *testing.T) { nil) // Command execution. - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000001", 123), - ).Return(environment, nil) - environment.EXPECT().GetBuildDirectory().Return(buildDirectory) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + false, + ).Return(buildDirectory, "0000000000000000", nil) filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().InstallHooks(filePool) + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000003", 345), - buildDirectory).Return(nil) + ).Return(nil) resourceUsage, err := ptypes.MarshalAny(&empty.Empty{}) require.NoError(t, err) - environment.EXPECT().Run(gomock.Any(), &runner.RunRequest{ + runner := mock.NewMockRunner(ctrl) + runner.EXPECT().Run(gomock.Any(), &runner_pb.RunRequest{ Arguments: []string{ "/usr/local/bin/clang", "-MD", @@ -476,19 +483,21 @@ func TestLocalBuildExecutorSuccess(t *testing.T) { "PATH": "/bin:/usr/bin", "PWD": "/proc/self/cwd", }, - WorkingDirectory: "", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }).Return(&runner.RunResponse{ + WorkingDirectory: "", + StdoutPath: "0000000000000000/stdout", + StderrPath: "0000000000000000/stderr", + InputRootDirectory: "0000000000000000/root", + }).Return(&runner_pb.RunResponse{ ExitCode: 0, ResourceUsage: []*any.Any{resourceUsage}, }, nil) - environment.EXPECT().Release() + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() clock := mock.NewMockClock(ctrl) clock.EXPECT().NewContextWithTimeout(gomock.Any(), time.Hour).DoAndReturn(func(parent context.Context, timeout time.Duration) (context.Context, context.CancelFunc) { return context.WithCancel(parent) }) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -583,10 +592,10 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { // Creation of output directory's parent and the directory itself // This will be the same directory used by the first output file - rootDirectory := mock.NewMockDirectory(ctrl) - rootDirectory.EXPECT().Mkdir("outputParent", os.FileMode(0777)).Return(nil) + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + inputRootDirectory.EXPECT().Mkdir("outputParent", os.FileMode(0777)).Return(nil) outputParentDirectory0 := mock.NewMockDirectoryCloser(ctrl) - rootDirectory.EXPECT().EnterDirectory("outputParent").Return(outputParentDirectory0, nil) + inputRootDirectory.EXPECT().EnterDirectory("outputParent").Return(outputParentDirectory0, nil) outputParentDirectory0.EXPECT().Close() outputParentDirectory0.EXPECT().Mkdir("output", os.FileMode(0777)).Return(nil) outputDirectory0 := mock.NewMockDirectoryCloser(ctrl) @@ -594,21 +603,21 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { outputDirectory0.EXPECT().Close() // Creation of the parent directory of second output file - rootDirectory.EXPECT().Mkdir("outputParent", os.FileMode(0777)).Return(syscall.EEXIST) + inputRootDirectory.EXPECT().Mkdir("outputParent", os.FileMode(0777)).Return(syscall.EEXIST) outputParentDirectory1 := mock.NewMockDirectoryCloser(ctrl) - rootDirectory.EXPECT().EnterDirectory("outputParent").Return(outputParentDirectory1, nil) + inputRootDirectory.EXPECT().EnterDirectory("outputParent").Return(outputParentDirectory1, nil) outputParentDirectory1.EXPECT().Mkdir("output", os.FileMode(0777)).Return(syscall.EEXIST) outputDirectory1 := mock.NewMockDirectoryCloser(ctrl) outputParentDirectory1.EXPECT().EnterDirectory("output").Return(outputDirectory1, nil) outputParentDirectory1.EXPECT().Close() outputDirectory1.EXPECT().Close() - outputDirectory1.EXPECT().Mkdir("build", os.FileMode(0777)).Return(nil) - buildDirectory := mock.NewMockDirectoryCloser(ctrl) - outputDirectory1.EXPECT().EnterDirectory("build").Return(buildDirectory, nil) - buildDirectory.EXPECT().Close() - buildDirectory.EXPECT().Mkdir("objects", os.FileMode(0777)).Return(nil) + outputDirectory1.EXPECT().Mkdir("foo", os.FileMode(0777)).Return(nil) + fooDirectory := mock.NewMockDirectoryCloser(ctrl) + outputDirectory1.EXPECT().EnterDirectory("foo").Return(fooDirectory, nil) + fooDirectory.EXPECT().Close() + fooDirectory.EXPECT().Mkdir("objects", os.FileMode(0777)).Return(nil) objectsDirectory := mock.NewMockDirectoryCloser(ctrl) - buildDirectory.EXPECT().EnterDirectory("objects").Return(objectsDirectory, nil) + fooDirectory.EXPECT().EnterDirectory("objects").Return(objectsDirectory, nil) objectsDirectory.EXPECT().Close() outputDirectory0.EXPECT().ReadDir().Return( @@ -626,10 +635,11 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) // Write operations against the Content Addressable Storage. - contentAddressableStorage.EXPECT().PutFile(ctx, rootDirectory, ".stdout.txt", gomock.Any()).Return( + buildDirectory := mock.NewMockBuildDirectory(ctrl) + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stdout", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000005", 567), nil) - contentAddressableStorage.EXPECT().PutFile(ctx, rootDirectory, ".stderr.txt", gomock.Any()).Return( + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stderr", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000006", 678), nil) contentAddressableStorage.EXPECT().PutFile(ctx, outputDirectory0, "hello.pic.d", gomock.Any()).Return( @@ -659,22 +669,23 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { nil) // Command execution. - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000001", 123), - ).Return(environment, nil) - environment.EXPECT().GetBuildDirectory().Return(rootDirectory) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + false, + ).Return(buildDirectory, ".", nil) filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().InstallHooks(filePool) + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000003", 345), - rootDirectory).Return(nil) + ).Return(nil) resourceUsage, err := ptypes.MarshalAny(&empty.Empty{}) require.NoError(t, err) - environment.EXPECT().Run(gomock.Any(), &runner.RunRequest{ + runner := mock.NewMockRunner(ctrl) + runner.EXPECT().Run(gomock.Any(), &runner_pb.RunRequest{ Arguments: []string{ "/usr/local/bin/clang", "-MD", @@ -689,19 +700,21 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { "PATH": "/bin:/usr/bin", "PWD": "/proc/self/cwd", }, - WorkingDirectory: "outputParent/output/build", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }).Return(&runner.RunResponse{ + WorkingDirectory: "outputParent/output/foo", + StdoutPath: "stdout", + StderrPath: "stderr", + InputRootDirectory: "root", + }).Return(&runner_pb.RunResponse{ ExitCode: 0, ResourceUsage: []*any.Any{resourceUsage}, }, nil) - environment.EXPECT().Release() + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() clock := mock.NewMockClock(ctrl) clock.EXPECT().NewContextWithTimeout(gomock.Any(), time.Hour).DoAndReturn(func(parent context.Context, timeout time.Duration) (context.Context, context.CancelFunc) { return context.WithCancel(parent) }) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -749,7 +762,7 @@ func TestLocalBuildExecutorWithWorkingDirectorySuccess(t *testing.T) { }, }, }, - WorkingDirectory: "outputParent/output/build", + WorkingDirectory: "outputParent/output/foo", }, }, metadata) @@ -801,10 +814,10 @@ func TestLocalBuildExecutorCachingInvalidTimeout(t *testing.T) { defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) // Execution should fail, as the number of nanoseconds in the // timeout is not within bounds. @@ -843,10 +856,10 @@ func TestLocalBuildExecutorCachingTimeoutTooHigh(t *testing.T) { defer ctrl.Finish() contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - environmentManager := mock.NewMockManager(ctrl) - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + runner := mock.NewMockRunner(ctrl) clock := mock.NewMockClock(ctrl) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) // The protocol states that we must deny requests that have a // timeout that is longer than the server's maximum. @@ -885,50 +898,54 @@ func TestLocalBuildExecutorTimeoutDuringExecution(t *testing.T) { defer ctrl.Finish() // Build directory. - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockBuildDirectory(ctrl) contentAddressableStorage := mock.NewMockContentAddressableStorage(ctrl) - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stdout.txt", gomock.Any()).Return( + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stdout", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000005", 567), nil) - contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, ".stderr.txt", gomock.Any()).Return( + contentAddressableStorage.EXPECT().PutFile(ctx, buildDirectory, "stderr", gomock.Any()).Return( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000006", 678), nil) // Build environment. - environmentManager := mock.NewMockManager(ctrl) - environment := mock.NewMockManagedEnvironment(ctrl) - environmentManager.EXPECT().Acquire( + buildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + buildDirectoryCreator.EXPECT().GetBuildDirectory( digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000001", 123), - ).Return(environment, nil) - environment.EXPECT().GetBuildDirectory().Return(buildDirectory) + false, + ).Return(buildDirectory, ".", nil) + filePool := mock.NewMockFilePool(ctrl) + buildDirectory.EXPECT().InstallHooks(filePool) // Input root creation. - inputRootPopulator := mock.NewMockInputRootPopulator(ctrl) - filePool := mock.NewMockFilePool(ctrl) - inputRootPopulator.EXPECT().PopulateInputRoot( + buildDirectory.EXPECT().Mkdir("root", os.FileMode(0777)) + inputRootDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectory.EXPECT().EnterBuildDirectory("root").Return(inputRootDirectory, nil) + inputRootDirectory.EXPECT().MergeDirectoryContents( ctx, - filePool, digest.MustNewDigest("ubuntu1804", "0000000000000000000000000000000000000000000000000000000000000003", 345), - buildDirectory).Return(nil) + ).Return(nil) // Simulate a timeout by running the command with a timeout of // zero seconds. This should cause an immediate build failure. - environment.EXPECT().Run(gomock.Any(), &runner.RunRequest{ + runner := mock.NewMockRunner(ctrl) + runner.EXPECT().Run(gomock.Any(), &runner_pb.RunRequest{ Arguments: []string{"clang"}, EnvironmentVariables: map[string]string{}, WorkingDirectory: "", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }).DoAndReturn(func(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { + StdoutPath: "stdout", + StderrPath: "stderr", + InputRootDirectory: "root", + }).DoAndReturn(func(ctx context.Context, request *runner_pb.RunRequest) (*runner_pb.RunResponse, error) { <-ctx.Done() return nil, util.StatusFromContext(ctx) }) - environment.EXPECT().Release() + inputRootDirectory.EXPECT().Close() + buildDirectory.EXPECT().Close() clock := mock.NewMockClock(ctrl) clock.EXPECT().NewContextWithTimeout(gomock.Any(), time.Hour).DoAndReturn(func(parent context.Context, timeout time.Duration) (context.Context, context.CancelFunc) { return context.WithTimeout(parent, 0) }) - localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, environmentManager, inputRootPopulator, clock, time.Hour, time.Hour) + localBuildExecutor := builder.NewLocalBuildExecutor(contentAddressableStorage, buildDirectoryCreator, runner, clock, time.Hour, time.Hour) metadata := make(chan *remoteworker.CurrentState_Executing, 10) executeResponse := localBuildExecutor.Execute( @@ -966,6 +983,3 @@ func TestLocalBuildExecutorTimeoutDuringExecution(t *testing.T) { Status: status.New(codes.DeadlineExceeded, "Failed to run command: context deadline exceeded").Proto(), }, executeResponse) } - -// TODO: Test aspects of execution not covered above (e.g., output -// directories, symlinks). diff --git a/pkg/builder/naive_input_root_populator.go b/pkg/builder/naive_build_directory.go similarity index 56% rename from pkg/builder/naive_input_root_populator.go rename to pkg/builder/naive_build_directory.go index 1cfa571c..59426b99 100644 --- a/pkg/builder/naive_input_root_populator.go +++ b/pkg/builder/naive_build_directory.go @@ -11,29 +11,47 @@ import ( "github.com/buildbarn/bb-storage/pkg/util" ) -type naiveInputRootPopulator struct { +type naiveBuildDirectory struct { + filesystem.DirectoryCloser contentAddressableStorage cas.ContentAddressableStorage } -// NewNaiveInputRootPopulator creates an input root populator that -// initializes the build directory of an action as simple as possible. -// Namely, it recursively loads all directories from the Content -// Addressable Storage (CAS) and requests that all of their files are -// copied into the build directory. +// NewNaiveBuildDirectory creates a BuildDirectory that is backed by a +// simple filesystem.Directory with all of the operations implemented in +// a naive way. Namely, MergeDirectoryContents() recursively loads all +// directories from the Content Addressable Storage (CAS) and requests +// that all of their files are copied into the build directory. // // This implementation is intended to be used in combination with // regular local file systems. The downside of such file systems is that // we cannot populate them on demand. All of the input files must be // present before invoking the build action. -func NewNaiveInputRootPopulator(contentAddressableStorage cas.ContentAddressableStorage) InputRootPopulator { - return &naiveInputRootPopulator{ +func NewNaiveBuildDirectory(directory filesystem.DirectoryCloser, contentAddressableStorage cas.ContentAddressableStorage) BuildDirectory { + return &naiveBuildDirectory{ + DirectoryCloser: directory, contentAddressableStorage: contentAddressableStorage, } } -func (ex *naiveInputRootPopulator) populateInputDirectory(ctx context.Context, digest digest.Digest, inputDirectory filesystem.Directory, components []string) error { +func (d *naiveBuildDirectory) EnterBuildDirectory(name string) (BuildDirectory, error) { + child, err := d.EnterDirectory(name) + if err != nil { + return nil, err + } + return &naiveBuildDirectory{ + DirectoryCloser: child, + contentAddressableStorage: d.contentAddressableStorage, + }, nil +} + +func (d *naiveBuildDirectory) InstallHooks(filePool re_filesystem.FilePool) { + // Simply ignore the provided hooks, as POSIX offers no way to + // install them. This means no quota enforcement is performed. +} + +func (d *naiveBuildDirectory) mergeDirectoryContents(ctx context.Context, digest digest.Digest, inputDirectory filesystem.Directory, components []string) error { // Obtain directory. - directory, err := ex.contentAddressableStorage.GetDirectory(ctx, digest) + directory, err := d.contentAddressableStorage.GetDirectory(ctx, digest) if err != nil { return util.StatusWrapf(err, "Failed to obtain input directory %#v", path.Join(components...)) } @@ -45,7 +63,7 @@ func (ex *naiveInputRootPopulator) populateInputDirectory(ctx context.Context, d if err != nil { return util.StatusWrapf(err, "Failed to extract digest for input file %#v", path.Join(childComponents...)) } - if err := ex.contentAddressableStorage.GetFile(ctx, childDigest, inputDirectory, file.Name, file.IsExecutable); err != nil { + if err := d.contentAddressableStorage.GetFile(ctx, childDigest, inputDirectory, file.Name, file.IsExecutable); err != nil { return util.StatusWrapf(err, "Failed to obtain input file %#v", path.Join(childComponents...)) } } @@ -62,7 +80,7 @@ func (ex *naiveInputRootPopulator) populateInputDirectory(ctx context.Context, d if err != nil { return util.StatusWrapf(err, "Failed to enter input directory %#v", path.Join(childComponents...)) } - err = ex.populateInputDirectory(ctx, childDigest, childDirectory, childComponents) + err = d.mergeDirectoryContents(ctx, childDigest, childDirectory, childComponents) childDirectory.Close() if err != nil { return err @@ -77,6 +95,6 @@ func (ex *naiveInputRootPopulator) populateInputDirectory(ctx context.Context, d return nil } -func (ex *naiveInputRootPopulator) PopulateInputRoot(ctx context.Context, filePool re_filesystem.FilePool, digest digest.Digest, inputDirectory filesystem.Directory) error { - return ex.populateInputDirectory(ctx, digest, inputDirectory, []string{"."}) +func (d *naiveBuildDirectory) MergeDirectoryContents(ctx context.Context, digest digest.Digest) error { + return d.mergeDirectoryContents(ctx, digest, d.DirectoryCloser, []string{"."}) } diff --git a/pkg/builder/naive_input_root_populator_test.go b/pkg/builder/naive_build_directory_test.go similarity index 81% rename from pkg/builder/naive_input_root_populator_test.go rename to pkg/builder/naive_build_directory_test.go index bed800bf..e1c8e476 100644 --- a/pkg/builder/naive_input_root_populator_test.go +++ b/pkg/builder/naive_build_directory_test.go @@ -16,7 +16,7 @@ import ( "google.golang.org/grpc/status" ) -func TestNaiveInputRootPopulatorSuccess(t *testing.T) { +func TestNaiveBuildDirectorySuccess(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -69,7 +69,7 @@ func TestNaiveInputRootPopulatorSuccess(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("directory", os.FileMode(0777)).Return(nil) nestedDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("directory").Return(nestedDirectory, nil) @@ -88,18 +88,15 @@ func TestNaiveInputRootPopulatorSuccess(t *testing.T) { "executable", true).Return(nil) buildDirectory.EXPECT().Symlink("executable", "link-to-executable").Return(nil) - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.NoError(t, err) } -func TestNaiveInputRootPopulatorInputRootNotInStorage(t *testing.T) { +func TestNaiveBuildDirectoryInputRootNotInStorage(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -108,19 +105,16 @@ func TestNaiveInputRootPopulatorInputRootNotInStorage(t *testing.T) { ctx, digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), ).Return(nil, status.Error(codes.Internal, "Storage is offline")) - buildDirectory := mock.NewMockDirectory(ctrl) - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.Internal, "Failed to obtain input directory \".\": Storage is offline")) } -func TestNaiveInputRootPopulatorMissingInputDirectoryDigest(t *testing.T) { +func TestNaiveBuildDirectoryMissingInputDirectoryDigest(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -149,23 +143,20 @@ func TestNaiveInputRootPopulatorMissingInputDirectoryDigest(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.InvalidArgument, "Failed to extract digest for input directory \"Hello/World\": No digest provided")) } -func TestNaiveInputRootPopulatorDirectoryCreationFailure(t *testing.T) { +func TestNaiveBuildDirectoryDirectoryCreationFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -198,24 +189,21 @@ func TestNaiveInputRootPopulatorDirectoryCreationFailure(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) helloDirectory.EXPECT().Mkdir("World", os.FileMode(0777)).Return(status.Error(codes.DataLoss, "Disk on fire")) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.DataLoss, "Failed to create input directory \"Hello/World\": Disk on fire")) } -func TestNaiveInputRootPopulatorDirectoryEnterFailure(t *testing.T) { +func TestNaiveBuildDirectoryDirectoryEnterDirectoryFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -248,25 +236,22 @@ func TestNaiveInputRootPopulatorDirectoryEnterFailure(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) helloDirectory.EXPECT().Mkdir("World", os.FileMode(0777)).Return(nil) helloDirectory.EXPECT().EnterDirectory("World").Return(nil, status.Error(codes.PermissionDenied, "Thou shalt not pass!")) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.PermissionDenied, "Failed to enter input directory \"Hello/World\": Thou shalt not pass!")) } -func TestNaiveInputRootPopulatorMissingInputFileDigest(t *testing.T) { +func TestNaiveBuildDirectoryMissingInputFileDigest(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -295,23 +280,20 @@ func TestNaiveInputRootPopulatorMissingInputFileDigest(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.InvalidArgument, "Failed to extract digest for input file \"Hello/World\": No digest provided")) } -func TestNaiveInputRootPopulatorFileCreationFailure(t *testing.T) { +func TestNaiveBuildDirectoryFileCreationFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -344,7 +326,7 @@ func TestNaiveInputRootPopulatorFileCreationFailure(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) @@ -355,18 +337,15 @@ func TestNaiveInputRootPopulatorFileCreationFailure(t *testing.T) { "World", false).Return(status.Error(codes.DataLoss, "Disk on fire")) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.DataLoss, "Failed to obtain input file \"Hello/World\": Disk on fire")) } -func TestNaiveInputRootPopulatorSymlinkCreationFailure(t *testing.T) { +func TestNaiveBuildDirectorySymlinkCreationFailure(t *testing.T) { ctrl, ctx := gomock.WithContext(context.Background(), t) defer ctrl.Finish() @@ -396,19 +375,16 @@ func TestNaiveInputRootPopulatorSymlinkCreationFailure(t *testing.T) { }, }, }, nil) - buildDirectory := mock.NewMockDirectory(ctrl) + buildDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().Mkdir("Hello", os.FileMode(0777)).Return(nil) helloDirectory := mock.NewMockDirectoryCloser(ctrl) buildDirectory.EXPECT().EnterDirectory("Hello").Return(helloDirectory, nil) helloDirectory.EXPECT().Symlink("/etc/passwd", "World").Return(status.Error(codes.Unimplemented, "This filesystem does not support symbolic links")) helloDirectory.EXPECT().Close() - inputRootPopulator := builder.NewNaiveInputRootPopulator(contentAddressableStorage) + inputRootPopulator := builder.NewNaiveBuildDirectory(buildDirectory, contentAddressableStorage) - filePool := mock.NewMockFilePool(ctrl) - err := inputRootPopulator.PopulateInputRoot( + err := inputRootPopulator.MergeDirectoryContents( ctx, - filePool, - digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42), - buildDirectory) + digest.MustNewDigest("netbsd", "7777777777777777777777777777777777777777777777777777777777777777", 42)) require.Equal(t, err, status.Error(codes.Unimplemented, "Failed to create input symlink \"Hello/World\": This filesystem does not support symbolic links")) } diff --git a/pkg/builder/root_build_directory_creator.go b/pkg/builder/root_build_directory_creator.go new file mode 100644 index 00000000..60caf99a --- /dev/null +++ b/pkg/builder/root_build_directory_creator.go @@ -0,0 +1,35 @@ +package builder + +import ( + "github.com/buildbarn/bb-storage/pkg/digest" +) + +type rootBuildDirectoryCreator struct { + buildDirectory BuildDirectory +} + +// NewRootBuildDirectoryCreator is a BuildDirectoryCreator that +// repeatedly hands out a single directory present on the current +// system. Additional decorators are used to run builds in +// subdirectories, so that build actions may run in parallel. +func NewRootBuildDirectoryCreator(buildDirectory BuildDirectory) BuildDirectoryCreator { + dc := &rootBuildDirectoryCreator{ + buildDirectory: rootBuildDirectory{ + BuildDirectory: buildDirectory, + }, + } + return dc +} + +func (dc *rootBuildDirectoryCreator) GetBuildDirectory(actionDigest digest.Digest, mayRunInParallel bool) (BuildDirectory, string, error) { + return dc.buildDirectory, ".", nil +} + +type rootBuildDirectory struct { + BuildDirectory +} + +func (d rootBuildDirectory) Close() error { + // Never call Close() on the root directory, as it will be reused. + return nil +} diff --git a/pkg/builder/root_build_directory_creator_test.go b/pkg/builder/root_build_directory_creator_test.go new file mode 100644 index 00000000..0e7a7aa7 --- /dev/null +++ b/pkg/builder/root_build_directory_creator_test.go @@ -0,0 +1,43 @@ +package builder_test + +import ( + "os" + "testing" + + "github.com/buildbarn/bb-remote-execution/internal/mock" + "github.com/buildbarn/bb-remote-execution/pkg/builder" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" +) + +func TestRootBuildDirectoryCreator(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockBuildDirectory := mock.NewMockBuildDirectory(ctrl) + buildDirectoryCreator := builder.NewRootBuildDirectoryCreator(mockBuildDirectory) + + // Run a simple build action that only performs an Mkdir() call. + // Once terminated, the underlying build directory should not be + // closed, as it is reused by the next build action. + mockBuildDirectory.EXPECT().Mkdir("hello", os.FileMode(0700)) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + true) + require.NoError(t, err) + require.Equal(t, ".", buildDirectoryPath) + require.NoError(t, buildDirectory.Mkdir("hello", os.FileMode(0700))) + buildDirectory.Close() + + // Run an action similar to the previous one. It should be + // applied against the same underlying build directory. + mockBuildDirectory.EXPECT().Mkdir("world", os.FileMode(0700)) + buildDirectory, buildDirectoryPath, err = buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("freebsd", "7609128715518308672067aab169e24944ead24e3d732aab8a8f0b7013a65564", 5), + true) + require.NoError(t, err) + require.Equal(t, ".", buildDirectoryPath) + require.NoError(t, buildDirectory.Mkdir("world", os.FileMode(0700))) + buildDirectory.Close() +} diff --git a/pkg/builder/shared_build_directory_creator.go b/pkg/builder/shared_build_directory_creator.go new file mode 100644 index 00000000..2c62281a --- /dev/null +++ b/pkg/builder/shared_build_directory_creator.go @@ -0,0 +1,114 @@ +package builder + +import ( + "log" + "path" + "strconv" + "sync/atomic" + + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/buildbarn/bb-storage/pkg/filesystem" + "github.com/buildbarn/bb-storage/pkg/util" + + "google.golang.org/grpc/codes" +) + +type sharedBuildDirectoryCreator struct { + base BuildDirectoryCreator + nextParallelActionID *uint64 +} + +// NewSharedBuildDirectoryCreator is an adapter for +// BuildDirectoryCreator that causes build actions to be executed inside +// a subdirectory within the build directory, as opposed to inside the +// build directory itself. The subdirectory is either named after the +// action digest of the build action or uses an incrementing number, +// based on whether collisions may occur. +// +// This adapter can be used to add concurrency to a single worker. When +// executing build actions in parallel, every build action needs its own +// build directory. +func NewSharedBuildDirectoryCreator(base BuildDirectoryCreator, nextParallelActionID *uint64) BuildDirectoryCreator { + return &sharedBuildDirectoryCreator{ + base: base, + nextParallelActionID: nextParallelActionID, + } +} + +func (dc *sharedBuildDirectoryCreator) GetBuildDirectory(actionDigest digest.Digest, mayRunInParallel bool) (BuildDirectory, string, error) { + parentDirectory, parentDirectoryPath, err := dc.base.GetBuildDirectory(actionDigest, mayRunInParallel) + if err != nil { + return nil, "", err + } + + // Determine the name of the subdirectory. + var childDirectoryName string + if mayRunInParallel { + // Multiple instances of this action may run in + // parallel, as the scheduler is not permitted to + // deduplicate them. This is likely caused by the + // 'do_not_cache' flag being set in the Action message. + // + // Number subdirectories incrementally to prevent + // collisions if multiple of them are scheduled on the + // same worker. + childDirectoryName = strconv.FormatUint(atomic.AddUint64(dc.nextParallelActionID, 1), 10) + } else { + // This action is guaranteed not to run in parallel, due + // to the scheduler being permitted to deduplicate + // execution requests. Use a directory name based on the + // action digest. This ensures that the working + // directory of the build action is deterministic, + // thereby increasing reproducibility. + // + // Only use a small number of characters from the digest + // to ensure the absolute path of the build directory + // remains short. This avoids reaching PATH_MAX and + // sockaddr_un::sun_path size limits for stronger digest + // functions. 16 characters is more than sufficient to + // prevent collisions. + childDirectoryName = actionDigest.GetHashString()[:16] + } + + // Create the subdirectory. + childDirectoryPath := path.Join(parentDirectoryPath, childDirectoryName) + if err := parentDirectory.Mkdir(childDirectoryName, 0777); err != nil { + parentDirectory.Close() + return nil, "", util.StatusWrapfWithCode(err, codes.Internal, "Failed to create build directory %#v", childDirectoryPath) + } + childDirectory, err := parentDirectory.EnterBuildDirectory(childDirectoryName) + if err != nil { + if err := parentDirectory.Remove(childDirectoryName); err != nil { + log.Printf("Failed to remove action digest build directory %#v upon failure to enter: %s", childDirectoryPath, err) + } + parentDirectory.Close() + return nil, "", util.StatusWrapfWithCode(err, codes.Internal, "Failed to enter build directory %#v", childDirectoryPath) + } + + return &sharedBuildDirectory{ + BuildDirectory: childDirectory, + parentDirectory: parentDirectory, + childDirectoryName: childDirectoryName, + childDirectoryPath: childDirectoryPath, + }, childDirectoryPath, nil +} + +type sharedBuildDirectory struct { + BuildDirectory + parentDirectory filesystem.DirectoryCloser + childDirectoryName string + childDirectoryPath string +} + +func (d *sharedBuildDirectory) Close() error { + err1 := d.BuildDirectory.Close() + err2 := d.parentDirectory.RemoveAll(d.childDirectoryName) + err3 := d.parentDirectory.Close() + if err1 != nil { + return util.StatusWrapf(err1, "Failed to close build directory %#v", d.childDirectoryPath) + } + if err2 != nil { + return util.StatusWrapfWithCode(err2, codes.Internal, "Failed to remove build directory %#v", d.childDirectoryPath) + } + return err3 +} diff --git a/pkg/builder/shared_build_directory_creator_test.go b/pkg/builder/shared_build_directory_creator_test.go new file mode 100644 index 00000000..f8c80394 --- /dev/null +++ b/pkg/builder/shared_build_directory_creator_test.go @@ -0,0 +1,236 @@ +package builder_test + +import ( + "os" + "testing" + + "github.com/buildbarn/bb-remote-execution/internal/mock" + "github.com/buildbarn/bb-remote-execution/pkg/builder" + "github.com/buildbarn/bb-storage/pkg/digest" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +func TestSharedBuildDirectoryCreatorGetBuildDirectoryFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Failure to create environment should simply be forwarded. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(nil, "", status.Error(codes.Internal, "No space left on device")) + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.Equal(t, status.Error(codes.Internal, "No space left on device"), err) +} + +func TestSharedBuildDirectoryCreatorMkdirFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Failure to create a build subdirectory is always an internal error. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)).Return( + status.Error(codes.AlreadyExists, "Directory already exists")) + baseBuildDirectory.EXPECT().Close() + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.Equal(t, status.Error(codes.Internal, "Failed to create build directory \"base-directory/e3b0c44298fc1c14\": Directory already exists"), err) +} + +func TestSharedBuildDirectoryCreatorEnterBuildDirectoryFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Failure to enter a build subdirectory is always an internal error. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)) + baseBuildDirectory.EXPECT().EnterBuildDirectory("e3b0c44298fc1c14").Return(nil, status.Error(codes.ResourceExhausted, "Out of file descriptors")) + baseBuildDirectory.EXPECT().Remove("e3b0c44298fc1c14") + baseBuildDirectory.EXPECT().Close() + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.Equal(t, status.Error(codes.Internal, "Failed to enter build directory \"base-directory/e3b0c44298fc1c14\": Out of file descriptors"), err) +} + +func TestSharedBuildDirectoryCreatorCloseChildFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Directory closure errors should be propagated. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)) + subDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectory.EXPECT().EnterBuildDirectory("e3b0c44298fc1c14").Return(subDirectory, nil) + subDirectory.EXPECT().Close().Return(status.Error(codes.Internal, "Bad file descriptor")) + baseBuildDirectory.EXPECT().RemoveAll("e3b0c44298fc1c14") + baseBuildDirectory.EXPECT().Close() + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.NoError(t, err) + require.Equal(t, "base-directory/e3b0c44298fc1c14", buildDirectoryPath) + require.Equal(t, status.Error(codes.Internal, "Failed to close build directory \"base-directory/e3b0c44298fc1c14\": Bad file descriptor"), buildDirectory.Close()) +} + +func TestSharedBuildDirectoryCreatorRemoveAllFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Directory removal errors should be propagated. Permission + // errors should be converted to internal errors, as they + // indicate problems with the infrastructure. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)) + subDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectory.EXPECT().EnterBuildDirectory("e3b0c44298fc1c14").Return(subDirectory, nil) + subDirectory.EXPECT().Close() + baseBuildDirectory.EXPECT().RemoveAll("e3b0c44298fc1c14").Return(status.Error(codes.PermissionDenied, "Directory is owned by another user")) + baseBuildDirectory.EXPECT().Close() + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.NoError(t, err) + require.Equal(t, "base-directory/e3b0c44298fc1c14", buildDirectoryPath) + require.Equal(t, status.Error(codes.Internal, "Failed to remove build directory \"base-directory/e3b0c44298fc1c14\": Directory is owned by another user"), buildDirectory.Close()) +} + +func TestSharedBuildDirectoryCreatorCloseParentFailure(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Directory closure errors on the parent should also be + // propagated, but there is no need to prefix any additional + // info. The base BuildDirectoryCreator will already be + // responsible for injecting more detailed errors. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)) + subDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectory.EXPECT().EnterBuildDirectory("e3b0c44298fc1c14").Return(subDirectory, nil) + subDirectory.EXPECT().Close() + baseBuildDirectory.EXPECT().RemoveAll("e3b0c44298fc1c14") + baseBuildDirectory.EXPECT().Close().Return(status.Error(codes.Internal, "Bad file descriptor")) + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.NoError(t, err) + require.Equal(t, "base-directory/e3b0c44298fc1c14", buildDirectoryPath) + require.Equal(t, status.Error(codes.Internal, "Bad file descriptor"), buildDirectory.Close()) +} + +func TestSharedBuildDirectoryCreatorSuccessNotParallel(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + // Successful build in a subdirectory for an action that does + // not run in parallel. The subdirectory name is based on the + // action digest. + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)) + subDirectory := mock.NewMockBuildDirectory(ctrl) + baseBuildDirectory.EXPECT().EnterBuildDirectory("e3b0c44298fc1c14").Return(subDirectory, nil) + subDirectory.EXPECT().Close() + baseBuildDirectory.EXPECT().RemoveAll("e3b0c44298fc1c14") + baseBuildDirectory.EXPECT().Close() + + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + buildDirectory, buildDirectoryPath, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + false) + require.NoError(t, err) + require.Equal(t, "base-directory/e3b0c44298fc1c14", buildDirectoryPath) + require.NoError(t, buildDirectory.Close()) +} + +func TestSharedBuildDirectoryCreatorMkdirSuccessParallel(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + baseBuildDirectoryCreator := mock.NewMockBuildDirectoryCreator(ctrl) + baseBuildDirectory := mock.NewMockBuildDirectory(ctrl) + var nextParallelActionID uint64 + buildDirectoryCreator := builder.NewSharedBuildDirectoryCreator(baseBuildDirectoryCreator, &nextParallelActionID) + + // Build directories for actions that run in parallel are simply + // named incrementally to prevent collisions. + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + true, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("1", os.FileMode(0777)).Return( + status.Error(codes.Internal, "Foo")) + baseBuildDirectory.EXPECT().Close() + _, _, err := buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + true) + require.Equal(t, status.Error(codes.Internal, "Failed to create build directory \"base-directory/1\": Foo"), err) + + baseBuildDirectoryCreator.EXPECT().GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + true, + ).Return(baseBuildDirectory, "base-directory", nil) + baseBuildDirectory.EXPECT().Mkdir("2", os.FileMode(0777)).Return( + status.Error(codes.Internal, "Foo")) + baseBuildDirectory.EXPECT().Close() + _, _, err = buildDirectoryCreator.GetBuildDirectory( + digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), + true) + require.Equal(t, status.Error(codes.Internal, "Failed to create build directory \"base-directory/2\": Foo"), err) +} diff --git a/pkg/environment/BUILD.bazel b/pkg/environment/BUILD.bazel deleted file mode 100644 index 0ac6aa9a..00000000 --- a/pkg/environment/BUILD.bazel +++ /dev/null @@ -1,53 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "go_default_library", - srcs = [ - "action_digest_subdirectory_manager.go", - "clean_build_directory_manager.go", - "concurrent_manager.go", - "environment.go", - "local_execution_environment.go", - "local_execution_environment_rss_bytes.go", - "local_execution_environment_rss_kibibytes.go", - "manager.go", - "remote_execution_environment.go", - "runner_server.go", - "singleton_manager.go", - "temp_directory_cleaning_manager.go", - ], - importpath = "github.com/buildbarn/bb-remote-execution/pkg/environment", - visibility = ["//visibility:public"], - deps = [ - "//pkg/proto/resourceusage:go_default_library", - "//pkg/proto/runner:go_default_library", - "@com_github_buildbarn_bb_storage//pkg/digest:go_default_library", - "@com_github_buildbarn_bb_storage//pkg/filesystem:go_default_library", - "@com_github_buildbarn_bb_storage//pkg/util:go_default_library", - "@com_github_golang_protobuf//ptypes:go_default_library_gen", - "@io_bazel_rules_go//proto/wkt:any_go_proto", - "@io_bazel_rules_go//proto/wkt:duration_go_proto", - "@io_bazel_rules_go//proto/wkt:empty_go_proto", - "@org_golang_google_grpc//:go_default_library", - "@org_golang_google_grpc//codes:go_default_library", - "@org_golang_google_grpc//status:go_default_library", - ], -) - -go_test( - name = "go_default_test", - srcs = [ - "action_digest_subdirectory_manager_test.go", - "clean_build_directory_manager_test.go", - ], - embed = [":go_default_library"], - deps = [ - "//internal/mock:go_default_library", - "//pkg/proto/runner:go_default_library", - "@com_github_buildbarn_bb_storage//pkg/digest:go_default_library", - "@com_github_golang_mock//gomock:go_default_library", - "@com_github_stretchr_testify//require:go_default_library", - "@org_golang_google_grpc//codes:go_default_library", - "@org_golang_google_grpc//status:go_default_library", - ], -) diff --git a/pkg/environment/action_digest_subdirectory_manager.go b/pkg/environment/action_digest_subdirectory_manager.go deleted file mode 100644 index 9538f449..00000000 --- a/pkg/environment/action_digest_subdirectory_manager.go +++ /dev/null @@ -1,99 +0,0 @@ -package environment - -import ( - "context" - "log" - "path" - - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/buildbarn/bb-storage/pkg/filesystem" - "github.com/buildbarn/bb-storage/pkg/util" - - "google.golang.org/grpc/codes" -) - -type actionDigestSubdirectoryManager struct { - base Manager -} - -// NewActionDigestSubdirectoryManager is an adapter for Manager that -// causes build actions to be executed inside a subdirectory within the -// build directory, as opposed to inside the build directory itself. The -// subdirectory is named after the action digest of the build action. -// -// This adapter can be used to add concurrency to a single worker. When -// executing build actions in parallel, every build action needs its own -// build directory. Instead of picking random pathnames or using a -// counter, using the action digest has the advantage of improving -// determinism in case absolute paths end up in build output. -func NewActionDigestSubdirectoryManager(base Manager) Manager { - return &actionDigestSubdirectoryManager{ - base: base, - } -} - -func (em *actionDigestSubdirectoryManager) Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) { - // Allocate underlying environment. - environment, err := em.base.Acquire(actionDigest) - if err != nil { - return nil, err - } - - // Create build directory within. Only use a small number of - // characters from the digest to ensure the absolute path of the - // build directory remains short. This avoids reaching PATH_MAX - // and sockaddr_un::sun_path size limits for stronger digest - // functions. 16 characters is more than sufficient to prevent - // collisions. - buildDirectory := environment.GetBuildDirectory() - subdirectoryName := actionDigest.GetHashString()[:16] - if err := buildDirectory.Mkdir(subdirectoryName, 0777); err != nil { - environment.Release() - return nil, util.StatusWrapfWithCode(err, codes.Internal, "Failed to create build subdirectory %#v", subdirectoryName) - } - subdirectory, err := buildDirectory.EnterDirectory(subdirectoryName) - if err != nil { - if err := buildDirectory.Remove(subdirectoryName); err != nil { - log.Print("Failed to remove action digest build directory upon failure to enter: ", err) - } - environment.Release() - return nil, util.StatusWrapfWithCode(err, codes.Internal, "Failed to enter build subdirectory %#v", subdirectoryName) - } - - return &actionDigestSubdirectoryEnvironment{ - base: environment, - subdirectory: subdirectory, - subdirectoryName: subdirectoryName, - }, nil -} - -type actionDigestSubdirectoryEnvironment struct { - base ManagedEnvironment - subdirectory filesystem.DirectoryCloser - subdirectoryName string -} - -func (e *actionDigestSubdirectoryEnvironment) GetBuildDirectory() filesystem.Directory { - return e.subdirectory -} - -func (e *actionDigestSubdirectoryEnvironment) Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { - // Prepend subdirectory name to working directory and log files of build action. - newRequest := *request - newRequest.WorkingDirectory = path.Join(e.subdirectoryName, newRequest.WorkingDirectory) - newRequest.StdoutPath = path.Join(e.subdirectoryName, newRequest.StdoutPath) - newRequest.StderrPath = path.Join(e.subdirectoryName, newRequest.StderrPath) - return e.base.Run(ctx, &newRequest) -} - -func (e *actionDigestSubdirectoryEnvironment) Release() { - // Remove subdirectory prior to releasing the environment. - if err := e.subdirectory.Close(); err != nil { - log.Printf("Failed to close build subdirectory %s: %s", e.subdirectoryName, err) - } - if err := e.base.GetBuildDirectory().RemoveAll(e.subdirectoryName); err != nil { - log.Printf("Failed to remove build subdirectory %s: %s", e.subdirectoryName, err) - } - e.base.Release() -} diff --git a/pkg/environment/action_digest_subdirectory_manager_test.go b/pkg/environment/action_digest_subdirectory_manager_test.go deleted file mode 100644 index 55c8513f..00000000 --- a/pkg/environment/action_digest_subdirectory_manager_test.go +++ /dev/null @@ -1,129 +0,0 @@ -package environment_test - -import ( - "context" - "os" - "testing" - - "github.com/buildbarn/bb-remote-execution/internal/mock" - "github.com/buildbarn/bb-remote-execution/pkg/environment" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/require" - - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" -) - -func TestActionDigestSubdirectoryManagerAcquireFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - // Failure to create environment should simply be forwarded. - baseManager := mock.NewMockManager(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(nil, status.Error(codes.Internal, "No space left on device")) - - manager := environment.NewActionDigestSubdirectoryManager(baseManager) - _, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.Equal(t, status.Error(codes.Internal, "No space left on device"), err) -} - -func TestActionDigestSubdirectoryManagerMkdirFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - // Failure to create a build subdirectory is always an internal error. - baseManager := mock.NewMockManager(ctrl) - baseEnvironment := mock.NewMockManagedEnvironment(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(baseEnvironment, nil) - rootDirectory := mock.NewMockDirectory(ctrl) - baseEnvironment.EXPECT().GetBuildDirectory().Return(rootDirectory).AnyTimes() - rootDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)).Return( - status.Error(codes.AlreadyExists, "Directory already exists")) - baseEnvironment.EXPECT().Release() - - manager := environment.NewActionDigestSubdirectoryManager(baseManager) - _, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.Equal(t, status.Error(codes.Internal, "Failed to create build subdirectory \"e3b0c44298fc1c14\": Directory already exists"), err) -} - -func TestActionDigestSubdirectoryManagerEnterFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - // Failure to enter a build subdirectory is always an internal error. - baseManager := mock.NewMockManager(ctrl) - baseEnvironment := mock.NewMockManagedEnvironment(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(baseEnvironment, nil) - rootDirectory := mock.NewMockDirectory(ctrl) - baseEnvironment.EXPECT().GetBuildDirectory().Return(rootDirectory).AnyTimes() - rootDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)).Return(nil) - rootDirectory.EXPECT().EnterDirectory("e3b0c44298fc1c14").Return(nil, status.Error(codes.ResourceExhausted, "Out of file descriptors")) - rootDirectory.EXPECT().Remove("e3b0c44298fc1c14").Return(nil) - baseEnvironment.EXPECT().Release() - - manager := environment.NewActionDigestSubdirectoryManager(baseManager) - _, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.Equal(t, status.Error(codes.Internal, "Failed to enter build subdirectory \"e3b0c44298fc1c14\": Out of file descriptors"), err) -} - -func TestActionDigestSubdirectoryManagerSuccess(t *testing.T) { - ctrl, ctx := gomock.WithContext(context.Background(), t) - defer ctrl.Finish() - - // Successful build in a subdirectory. - baseManager := mock.NewMockManager(ctrl) - baseEnvironment := mock.NewMockManagedEnvironment(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(baseEnvironment, nil) - rootDirectory := mock.NewMockDirectory(ctrl) - baseEnvironment.EXPECT().GetBuildDirectory().Return(rootDirectory).AnyTimes() - rootDirectory.EXPECT().Mkdir("e3b0c44298fc1c14", os.FileMode(0777)).Return(nil) - subDirectory := mock.NewMockDirectoryCloser(ctrl) - rootDirectory.EXPECT().EnterDirectory("e3b0c44298fc1c14").Return(subDirectory, nil) - baseEnvironment.EXPECT().Run(ctx, &runner.RunRequest{ - Arguments: []string{"ls", "-l"}, - EnvironmentVariables: map[string]string{ - "PATH": "/bin", - }, - WorkingDirectory: "e3b0c44298fc1c14/some/sub/directory", - StdoutPath: "e3b0c44298fc1c14/.stdout.txt", - StderrPath: "e3b0c44298fc1c14/.stderr.txt", - }).Return(&runner.RunResponse{ - ExitCode: 123, - }, nil) - subDirectory.EXPECT().Close() - rootDirectory.EXPECT().RemoveAll("e3b0c44298fc1c14").Return(nil) - baseEnvironment.EXPECT().Release() - - manager := environment.NewActionDigestSubdirectoryManager(baseManager) - environment, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.NoError(t, err) - require.Equal(t, subDirectory, environment.GetBuildDirectory()) - response, err := environment.Run(ctx, &runner.RunRequest{ - Arguments: []string{"ls", "-l"}, - EnvironmentVariables: map[string]string{ - "PATH": "/bin", - }, - WorkingDirectory: "some/sub/directory", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }) - require.NoError(t, err) - require.Equal(t, &runner.RunResponse{ - ExitCode: 123, - }, response) - environment.Release() -} diff --git a/pkg/environment/clean_build_directory_manager.go b/pkg/environment/clean_build_directory_manager.go deleted file mode 100644 index ef730e77..00000000 --- a/pkg/environment/clean_build_directory_manager.go +++ /dev/null @@ -1,36 +0,0 @@ -package environment - -import ( - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/buildbarn/bb-storage/pkg/util" - - "google.golang.org/grpc/codes" -) - -type cleanBuildDirectoryManager struct { - base Manager -} - -// NewCleanBuildDirectoryManager is an adapter for Manager that upon -// acquistion empties out the build directory. This ensures that the -// build action is executed in a clean environment. -func NewCleanBuildDirectoryManager(base Manager) Manager { - return &cleanBuildDirectoryManager{ - base: base, - } -} - -func (em *cleanBuildDirectoryManager) Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) { - // Allocate underlying environment. - environment, err := em.base.Acquire(actionDigest) - if err != nil { - return nil, err - } - - // Remove all contents prior to use. - if err := environment.GetBuildDirectory().RemoveAllChildren(); err != nil { - environment.Release() - return nil, util.StatusWrapWithCode(err, codes.Internal, "Failed to clean build directory prior to build") - } - return environment, nil -} diff --git a/pkg/environment/clean_build_directory_manager_test.go b/pkg/environment/clean_build_directory_manager_test.go deleted file mode 100644 index 114ded2f..00000000 --- a/pkg/environment/clean_build_directory_manager_test.go +++ /dev/null @@ -1,101 +0,0 @@ -package environment_test - -import ( - "context" - "testing" - - "github.com/buildbarn/bb-remote-execution/internal/mock" - "github.com/buildbarn/bb-remote-execution/pkg/environment" - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/require" - - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" -) - -func TestCleanBuildDirectoryManagerAcquireFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - // Failure to create environment should simply be forwarded. - baseManager := mock.NewMockManager(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(nil, status.Error(codes.Internal, "No space left on device")) - - manager := environment.NewCleanBuildDirectoryManager(baseManager) - _, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.Equal(t, status.Error(codes.Internal, "No space left on device"), err) -} - -func TestCleanBuildDirectoryManagerRemoveAllChildrenFailure(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - // Failure to clean the build subdirectory is always an internal error. - baseManager := mock.NewMockManager(ctrl) - baseEnvironment := mock.NewMockManagedEnvironment(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(baseEnvironment, nil) - rootDirectory := mock.NewMockDirectory(ctrl) - baseEnvironment.EXPECT().GetBuildDirectory().Return(rootDirectory).AnyTimes() - rootDirectory.EXPECT().RemoveAllChildren().Return( - status.Error(codes.PermissionDenied, "You don't have permissions to remove files from disk")) - baseEnvironment.EXPECT().Release() - - manager := environment.NewCleanBuildDirectoryManager(baseManager) - _, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.Equal(t, status.Error(codes.Internal, "Failed to clean build directory prior to build: You don't have permissions to remove files from disk"), err) -} - -func TestCleanBuildDirectoryManagerSuccess(t *testing.T) { - ctrl, ctx := gomock.WithContext(context.Background(), t) - defer ctrl.Finish() - - // Successful build in a clean build directory. - baseManager := mock.NewMockManager(ctrl) - baseEnvironment := mock.NewMockManagedEnvironment(ctrl) - baseManager.EXPECT().Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0), - ).Return(baseEnvironment, nil) - rootDirectory := mock.NewMockDirectory(ctrl) - baseEnvironment.EXPECT().GetBuildDirectory().Return(rootDirectory).AnyTimes() - rootDirectory.EXPECT().RemoveAllChildren().Return(nil) - baseEnvironment.EXPECT().Run(ctx, &runner.RunRequest{ - Arguments: []string{"ls", "-l"}, - EnvironmentVariables: map[string]string{ - "PATH": "/bin", - }, - WorkingDirectory: "some/sub/directory", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }).Return(&runner.RunResponse{ - ExitCode: 123, - }, nil) - baseEnvironment.EXPECT().Release() - - manager := environment.NewCleanBuildDirectoryManager(baseManager) - environment, err := manager.Acquire( - digest.MustNewDigest("debian8", "e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855", 0)) - require.NoError(t, err) - require.Equal(t, rootDirectory, environment.GetBuildDirectory()) - response, err := environment.Run(ctx, &runner.RunRequest{ - Arguments: []string{"ls", "-l"}, - EnvironmentVariables: map[string]string{ - "PATH": "/bin", - }, - WorkingDirectory: "some/sub/directory", - StdoutPath: ".stdout.txt", - StderrPath: ".stderr.txt", - }) - require.NoError(t, err) - require.Equal(t, &runner.RunResponse{ - ExitCode: 123, - }, response) - environment.Release() -} diff --git a/pkg/environment/concurrent_manager.go b/pkg/environment/concurrent_manager.go deleted file mode 100644 index 20a115f7..00000000 --- a/pkg/environment/concurrent_manager.go +++ /dev/null @@ -1,65 +0,0 @@ -package environment - -import ( - "log" - "sync" - - "github.com/buildbarn/bb-storage/pkg/digest" -) - -type concurrentManager struct { - base Manager - lock sync.Mutex - refcount uint - environment ManagedEnvironment -} - -// NewConcurrentManager is an adapter for Manager that causes concurrent -// acquisition of environments to use the same underlying Environment. -// By using reference counting, the underlying Environment is released -// only when the last consumer releases. -// -// This adapter is commonly used in combination with other adapters like -// ActionDigestSubdirectoryManager to ensure that a single build -// directory can be shared by concurrent build actions. -func NewConcurrentManager(base Manager) Manager { - return &concurrentManager{ - base: base, - } -} - -func (em *concurrentManager) Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) { - em.lock.Lock() - defer em.lock.Unlock() - if em.refcount == 0 { - // No environment acquired yet. Call into the underlying manager. - environment, err := em.base.Acquire(actionDigest) - if err != nil { - return nil, err - } - em.environment = &concurrentEnvironment{ - ManagedEnvironment: environment, - manager: em, - } - } - em.refcount++ - return em.environment, nil -} - -type concurrentEnvironment struct { - ManagedEnvironment - manager *concurrentManager -} - -func (e *concurrentEnvironment) Release() { - e.manager.lock.Lock() - defer e.manager.lock.Unlock() - if e.manager.refcount == 0 { - log.Fatal("Attempted to release an already released environment") - } - e.manager.refcount-- - if e.manager.refcount == 0 { - // Last consumer released. Release the underlying environment. - e.ManagedEnvironment.Release() - } -} diff --git a/pkg/environment/environment.go b/pkg/environment/environment.go deleted file mode 100644 index 46762c71..00000000 --- a/pkg/environment/environment.go +++ /dev/null @@ -1,24 +0,0 @@ -package environment - -import ( - "context" - - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/filesystem" -) - -// Environment represents a context in which build commands may be -// invoked. Examples of environments may include Docker containers, -// simple chroots, local execution, etc., etc. -type Environment interface { - // Run a command within the environment, with a given set of - // arguments, environment variables, a working directory - // relative to the build directory and a pair of pathnames for - // writing stdout/stderr diagnostics output. - Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) - - // GetBuildDirectory returns a handle to a directory in which a - // BuildExecutor may place the input files of the build step and - // where output files created by the build action are stored. - GetBuildDirectory() filesystem.Directory -} diff --git a/pkg/environment/manager.go b/pkg/environment/manager.go deleted file mode 100644 index 168b9301..00000000 --- a/pkg/environment/manager.go +++ /dev/null @@ -1,24 +0,0 @@ -package environment - -import ( - "github.com/buildbarn/bb-storage/pkg/digest" -) - -// Manager is a factory for Environments in which build actions are run. An -// Manager has access to platform properties passed to the command to be -// executed. This may allow the Manager to, for example, download container -// images or set up simulators/emulators. -type Manager interface { - Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) -} - -// ManagedEnvironment is an environment that is owned by a Manager. -// After use, it must be released, so that resources associated with it -// (e.g., a running container, a build directory) may be destroyed. -type ManagedEnvironment interface { - Environment - - // Release the Environment back to the EnvironmentManager, - // causing any input/output files to be discarded. - Release() -} diff --git a/pkg/environment/remote_execution_environment.go b/pkg/environment/remote_execution_environment.go deleted file mode 100644 index 855cb1b3..00000000 --- a/pkg/environment/remote_execution_environment.go +++ /dev/null @@ -1,32 +0,0 @@ -package environment - -import ( - "context" - - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/filesystem" - - "google.golang.org/grpc" -) - -type remoteExecutionEnvironment struct { - buildDirectory filesystem.Directory - runner runner.RunnerClient -} - -// NewRemoteExecutionEnvironment returns an Environment capable of -// forwarding commands to a GRPC service. -func NewRemoteExecutionEnvironment(client *grpc.ClientConn, buildDirectory filesystem.Directory) Environment { - return &remoteExecutionEnvironment{ - buildDirectory: buildDirectory, - runner: runner.NewRunnerClient(client), - } -} - -func (e *remoteExecutionEnvironment) GetBuildDirectory() filesystem.Directory { - return e.buildDirectory -} - -func (e *remoteExecutionEnvironment) Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { - return e.runner.Run(ctx, request) -} diff --git a/pkg/environment/runner_server.go b/pkg/environment/runner_server.go deleted file mode 100644 index 5479b730..00000000 --- a/pkg/environment/runner_server.go +++ /dev/null @@ -1,34 +0,0 @@ -package environment - -import ( - "context" - - "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/golang/protobuf/ptypes/empty" -) - -type runnerServer struct { - manager Manager -} - -// NewRunnerServer returns a RunnerServer that runs every action in its -// own acquired Environment. -func NewRunnerServer(manager Manager) runner.RunnerServer { - return &runnerServer{ - manager: manager, - } -} - -func (rs *runnerServer) CheckReadiness(ctx context.Context, request *empty.Empty) (*empty.Empty, error) { - return &empty.Empty{}, nil -} - -func (rs *runnerServer) Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { - env, err := rs.manager.Acquire(digest.BadDigest) - if err != nil { - return nil, err - } - defer env.Release() - return env.Run(ctx, request) -} diff --git a/pkg/environment/singleton_manager.go b/pkg/environment/singleton_manager.go deleted file mode 100644 index 34030767..00000000 --- a/pkg/environment/singleton_manager.go +++ /dev/null @@ -1,39 +0,0 @@ -package environment - -import ( - "github.com/buildbarn/bb-storage/pkg/digest" -) - -type singletonManager struct { - environment chan ManagedEnvironment -} - -// NewSingletonManager is a simple Manager that always returns the same -// Environment. This is typically used in combination with -// NewLocalExecutionEnvironment or NewRemoteExecutionManager to force -// that all build actions are executed using the same method. -func NewSingletonManager(environment Environment) Manager { - em := &singletonManager{ - environment: make(chan ManagedEnvironment, 1), - } - em.environment <- &singletonEnvironment{ - Environment: environment, - manager: em, - } - return em -} - -func (em *singletonManager) Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) { - return <-em.environment, nil -} - -type singletonEnvironment struct { - Environment - manager *singletonManager -} - -func (e *singletonEnvironment) Release() { - // Never call Release() on the underlying environment, as it - // will be reused. - e.manager.environment <- e -} diff --git a/pkg/environment/temp_directory_cleaning_manager.go b/pkg/environment/temp_directory_cleaning_manager.go deleted file mode 100644 index 3675b1ef..00000000 --- a/pkg/environment/temp_directory_cleaning_manager.go +++ /dev/null @@ -1,34 +0,0 @@ -package environment - -import ( - "github.com/buildbarn/bb-storage/pkg/digest" - "github.com/buildbarn/bb-storage/pkg/filesystem" - "github.com/buildbarn/bb-storage/pkg/util" - - "google.golang.org/grpc/codes" -) - -type tempDirectoryCleaningManager struct { - base Manager - tempDirectory filesystem.Directory -} - -// NewTempDirectoryCleaningManager is an adapter for Manager that upon -// acquisition empties out a temporary directory (e.g. /tmp, /var/tmp, etc). -// This ensures that the build action is executed in a clean environment. -func NewTempDirectoryCleaningManager(base Manager, tempDirectory filesystem.Directory) Manager { - return &tempDirectoryCleaningManager{ - base: base, - tempDirectory: tempDirectory, - } -} - -func (em *tempDirectoryCleaningManager) Acquire(actionDigest digest.Digest) (ManagedEnvironment, error) { - // Remove all contents prior to use. - if err := em.tempDirectory.RemoveAllChildren(); err != nil { - return nil, util.StatusWrapWithCode(err, codes.Internal, "Failed to clean temporary directory prior to build") - } - - // Allocate underlying environment. - return em.base.Acquire(actionDigest) -} diff --git a/pkg/proto/runner/runner.proto b/pkg/proto/runner/runner.proto index 6959f9e6..df032863 100644 --- a/pkg/proto/runner/runner.proto +++ b/pkg/proto/runner/runner.proto @@ -31,7 +31,7 @@ message RunRequest { // Environment variables that need to be set. map environment_variables = 2; - // Working directory, relative to the build directory. + // Working directory, relative to the input root directory. string working_directory = 3; // Path where data written over stdout should be stored, relative to @@ -41,6 +41,9 @@ message RunRequest { // Path where data written over stderr should be stored, relative to // the build directory. string stderr_path = 5; + + // Path of the input root, relative to the build directory. + string input_root_directory = 6; } message RunResponse { diff --git a/pkg/runner/BUILD.bazel b/pkg/runner/BUILD.bazel new file mode 100644 index 00000000..3fa42206 --- /dev/null +++ b/pkg/runner/BUILD.bazel @@ -0,0 +1,30 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "go_default_library", + srcs = [ + "local_runner.go", + "local_runner_rss_bytes.go", + "local_runner_rss_kibibytes.go", + "remote_runner.go", + "runner.go", + "runner_server.go", + "temporary_directory_cleaning_runner.go", + ], + importpath = "github.com/buildbarn/bb-remote-execution/pkg/runner", + visibility = ["//visibility:public"], + deps = [ + "//pkg/proto/resourceusage:go_default_library", + "//pkg/proto/runner:go_default_library", + "//pkg/sync:go_default_library", + "@com_github_buildbarn_bb_storage//pkg/filesystem:go_default_library", + "@com_github_buildbarn_bb_storage//pkg/util:go_default_library", + "@com_github_golang_protobuf//ptypes:go_default_library_gen", + "@io_bazel_rules_go//proto/wkt:any_go_proto", + "@io_bazel_rules_go//proto/wkt:duration_go_proto", + "@io_bazel_rules_go//proto/wkt:empty_go_proto", + "@org_golang_google_grpc//:go_default_library", + "@org_golang_google_grpc//codes:go_default_library", + "@org_golang_google_grpc//status:go_default_library", + ], +) diff --git a/pkg/environment/local_execution_environment.go b/pkg/runner/local_runner.go similarity index 77% rename from pkg/environment/local_execution_environment.go rename to pkg/runner/local_runner.go index bf963f5b..e48379eb 100644 --- a/pkg/environment/local_execution_environment.go +++ b/pkg/runner/local_runner.go @@ -1,4 +1,4 @@ -package environment +package runner import ( "context" @@ -20,32 +20,28 @@ import ( "google.golang.org/grpc/status" ) -type localExecutionEnvironment struct { - buildDirectory filesystem.Directory - buildPath string +type localRunner struct { + buildDirectory filesystem.Directory + buildDirectoryPath string } -// NewLocalExecutionEnvironment returns an Environment capable of running -// commands on the local system directly. -func NewLocalExecutionEnvironment(buildDirectory filesystem.Directory, buildPath string) Environment { - return &localExecutionEnvironment{ - buildDirectory: buildDirectory, - buildPath: buildPath, +// NewLocalRunner returns a Runner capable of running commands on the +// local system directly. +func NewLocalRunner(buildDirectory filesystem.Directory, buildDirectoryPath string) Runner { + return &localRunner{ + buildDirectory: buildDirectory, + buildDirectoryPath: buildDirectoryPath, } } -func (e *localExecutionEnvironment) GetBuildDirectory() filesystem.Directory { - return e.buildDirectory -} - -func (e *localExecutionEnvironment) openLog(logPath string) (filesystem.FileAppender, error) { +func (r *localRunner) openLog(logPath string) (filesystem.FileAppender, error) { components := strings.FieldsFunc(logPath, func(r rune) bool { return r == '/' }) if len(components) < 1 { return nil, status.Error(codes.InvalidArgument, "Insufficient pathname components in filename") } // Traverse to directory where log should be created. - d := filesystem.NopDirectoryCloser(e.buildDirectory) + d := filesystem.NopDirectoryCloser(r.buildDirectory) for n, component := range components[:len(components)-1] { d2, err := d.EnterDirectory(component) d.Close() @@ -68,26 +64,26 @@ func convertTimeval(t syscall.Timeval) *duration.Duration { } } -func (e *localExecutionEnvironment) Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { +func (r *localRunner) Run(ctx context.Context, request *runner.RunRequest) (*runner.RunResponse, error) { if len(request.Arguments) < 1 { return nil, status.Error(codes.InvalidArgument, "Insufficient number of command arguments") } cmd := exec.CommandContext(ctx, request.Arguments[0], request.Arguments[1:]...) - // TODO(edsch): Convert workingDirectory to use platform - // specific path delimiter. - cmd.Dir = filepath.Join(e.buildPath, request.WorkingDirectory) + // TODO: Convert WorkingDirectory to use platform specific path + // delimiters. + cmd.Dir = filepath.Join(r.buildDirectoryPath, request.InputRootDirectory, request.WorkingDirectory) for name, value := range request.EnvironmentVariables { cmd.Env = append(cmd.Env, name+"="+value) } // Open output files for logging. - stdout, err := e.openLog(request.StdoutPath) + stdout, err := r.openLog(request.StdoutPath) if err != nil { return nil, util.StatusWrap(err, "Failed to open stdout") } cmd.Stdout = stdout - stderr, err := e.openLog(request.StderrPath) + stderr, err := r.openLog(request.StderrPath) if err != nil { stdout.Close() return nil, util.StatusWrap(err, "Failed to open stderr") diff --git a/pkg/environment/local_execution_environment_rss_bytes.go b/pkg/runner/local_runner_rss_bytes.go similarity index 89% rename from pkg/environment/local_execution_environment_rss_bytes.go rename to pkg/runner/local_runner_rss_bytes.go index ca2bb004..f7f9d4eb 100644 --- a/pkg/environment/local_execution_environment_rss_bytes.go +++ b/pkg/runner/local_runner_rss_bytes.go @@ -1,6 +1,6 @@ // +build darwin -package environment +package runner const ( // On macOS, the getrusage(2) man page documents that the diff --git a/pkg/environment/local_execution_environment_rss_kibibytes.go b/pkg/runner/local_runner_rss_kibibytes.go similarity index 92% rename from pkg/environment/local_execution_environment_rss_kibibytes.go rename to pkg/runner/local_runner_rss_kibibytes.go index 20192b52..996be2af 100644 --- a/pkg/environment/local_execution_environment_rss_kibibytes.go +++ b/pkg/runner/local_runner_rss_kibibytes.go @@ -1,6 +1,6 @@ // +build freebsd linux -package environment +package runner const ( // On Linux and FreeBSD, the getrusage(2) man pages document diff --git a/pkg/runner/remote_runner.go b/pkg/runner/remote_runner.go new file mode 100644 index 00000000..b7e42e0b --- /dev/null +++ b/pkg/runner/remote_runner.go @@ -0,0 +1,25 @@ +package runner + +import ( + "context" + + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + + "google.golang.org/grpc" +) + +type remoteRunner struct { + runner runner_pb.RunnerClient +} + +// NewRemoteRunner creates a Runner that forwards requests to a remote +// gRPC service (typically an instance of bb_runner). +func NewRemoteRunner(connection *grpc.ClientConn) Runner { + return &remoteRunner{ + runner: runner_pb.NewRunnerClient(connection), + } +} + +func (r *remoteRunner) Run(ctx context.Context, request *runner_pb.RunRequest) (*runner_pb.RunResponse, error) { + return r.runner.Run(ctx, request) +} diff --git a/pkg/runner/runner.go b/pkg/runner/runner.go new file mode 100644 index 00000000..5b25e10d --- /dev/null +++ b/pkg/runner/runner.go @@ -0,0 +1,13 @@ +package runner + +import ( + "context" + + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" +) + +// Runner of build actions. LocalBuildExecutor calls into this interface +// to spawn the process described in the Command message. +type Runner interface { + Run(ctx context.Context, request *runner_pb.RunRequest) (*runner_pb.RunResponse, error) +} diff --git a/pkg/runner/runner_server.go b/pkg/runner/runner_server.go new file mode 100644 index 00000000..d2d7f4ab --- /dev/null +++ b/pkg/runner/runner_server.go @@ -0,0 +1,28 @@ +package runner + +import ( + "context" + + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + "github.com/golang/protobuf/ptypes/empty" +) + +type runnerServer struct { + runner Runner +} + +// NewRunnerServer returns a gRPC RunnerServer that forwards every call +// to a plain Runner. +func NewRunnerServer(runner Runner) runner_pb.RunnerServer { + return &runnerServer{ + runner: runner, + } +} + +func (rs *runnerServer) CheckReadiness(ctx context.Context, request *empty.Empty) (*empty.Empty, error) { + return &empty.Empty{}, nil +} + +func (rs *runnerServer) Run(ctx context.Context, request *runner_pb.RunRequest) (*runner_pb.RunResponse, error) { + return rs.runner.Run(ctx, request) +} diff --git a/pkg/runner/temporary_directory_cleaning_runner.go b/pkg/runner/temporary_directory_cleaning_runner.go new file mode 100644 index 00000000..0a35339d --- /dev/null +++ b/pkg/runner/temporary_directory_cleaning_runner.go @@ -0,0 +1,43 @@ +package runner + +import ( + "context" + + runner_pb "github.com/buildbarn/bb-remote-execution/pkg/proto/runner" + "github.com/buildbarn/bb-remote-execution/pkg/sync" + "github.com/buildbarn/bb-storage/pkg/filesystem" + "github.com/buildbarn/bb-storage/pkg/util" + + "google.golang.org/grpc/codes" +) + +type temporaryDirectoryCleaningRunner struct { + base Runner + directory filesystem.Directory + directoryPath string + + initializer sync.Initializer +} + +// NewTemporaryDirectoryCleaningRunner is an adapter for Runner that +// empties out a temporary directory (e.g. /tmp, /var/tmp, etc) when +// transitioning from idle to being used. This ensures that build +// actions are executed in a clean environment. +func NewTemporaryDirectoryCleaningRunner(base Runner, directory filesystem.Directory, directoryPath string) Runner { + return &temporaryDirectoryCleaningRunner{ + base: base, + directory: directory, + directoryPath: directoryPath, + } +} + +func (r *temporaryDirectoryCleaningRunner) Run(ctx context.Context, request *runner_pb.RunRequest) (*runner_pb.RunResponse, error) { + if err := r.initializer.Acquire(r.directory.RemoveAllChildren); err != nil { + return nil, util.StatusWrapfWithCode(err, codes.Internal, "Failed to clean temporary directory %#v prior to build", r.directoryPath) + } + // TODO: Should we also trigger a cleanup when going back to + // idle, so that files don't remain on disk longer than + // necessary? + defer r.initializer.Release() + return r.base.Run(ctx, request) +}