Skip to content

Commit acc90ac

Browse files
committed
Remove MultiPlatform Process abstractions
We don't actually use these anywhere - instead, always operate on a single platform-specific or platform-independent Process.
1 parent 3eb1712 commit acc90ac

File tree

17 files changed

+173
-380
lines changed

17 files changed

+173
-380
lines changed

src/python/pants/backend/python/util_rules/pex.py

+4-8
Original file line numberDiff line numberDiff line change
@@ -48,13 +48,7 @@
4848
PathGlobs,
4949
)
5050
from pants.engine.platform import Platform
51-
from pants.engine.process import (
52-
BashBinary,
53-
MultiPlatformProcess,
54-
Process,
55-
ProcessCacheScope,
56-
ProcessResult,
57-
)
51+
from pants.engine.process import BashBinary, Process, ProcessCacheScope, ProcessResult
5852
from pants.engine.rules import Get, MultiGet, collect_rules, rule
5953
from pants.python.python_repos import PythonRepos
6054
from pants.python.python_setup import InvalidLockfileBehavior, PythonSetup
@@ -559,10 +553,12 @@ async def build_pex_component(
559553
),
560554
)
561555

556+
process = dataclasses.replace(process, platform=platform)
557+
562558
# NB: Building a Pex is platform dependent, so in order to get a PEX that we can use locally
563559
# without cross-building, we specify that our PEX command should be run on the current local
564560
# platform.
565-
result = await Get(ProcessResult, MultiPlatformProcess({platform: process}))
561+
result = await Get(ProcessResult, Process, process)
566562

567563
if pex_runtime_env.verbosity > 0:
568564
log_output = result.stderr.decode()

src/python/pants/engine/internals/engine_test.py

+4-4
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@
3131
from pants.engine.internals.scheduler import ExecutionError, SchedulerSession
3232
from pants.engine.internals.scheduler_test_base import SchedulerTestBase
3333
from pants.engine.platform import rules as platform_rules
34-
from pants.engine.process import MultiPlatformProcess, Process, ProcessCacheScope, ProcessResult
34+
from pants.engine.process import Process, ProcessCacheScope, ProcessResult
3535
from pants.engine.process import rules as process_rules
3636
from pants.engine.rules import Get, MultiGet, rule
3737
from pants.engine.streaming_workunit_handler import (
@@ -611,7 +611,7 @@ async def a_rule() -> TrueResult:
611611
description="always true",
612612
cache_scope=ProcessCacheScope.PER_SESSION,
613613
)
614-
_ = await Get(ProcessResult, MultiPlatformProcess({None: proc}))
614+
_ = await Get(ProcessResult, Process, proc)
615615
return TrueResult()
616616

617617
scheduler, tracker, handler = self._fixture_for_rules(
@@ -753,7 +753,7 @@ def test_process_digests_on_streaming_workunits(
753753
assert tracker.finished
754754
finished = list(itertools.chain.from_iterable(tracker.finished_workunit_chunks))
755755

756-
process_workunit = next(item for item in finished if item["name"] == "multi_platform_process")
756+
process_workunit = next(item for item in finished if item["name"] == "process")
757757
assert process_workunit is not None
758758
stdout_digest = process_workunit["artifacts"]["stdout_digest"]
759759
stderr_digest = process_workunit["artifacts"]["stderr_digest"]
@@ -781,7 +781,7 @@ def test_process_digests_on_streaming_workunits(
781781

782782
assert tracker.finished
783783
finished = list(itertools.chain.from_iterable(tracker.finished_workunit_chunks))
784-
process_workunit = next(item for item in finished if item["name"] == "multi_platform_process")
784+
process_workunit = next(item for item in finished if item["name"] == "process")
785785

786786
assert process_workunit is not None
787787
stdout_digest = process_workunit["artifacts"]["stdout_digest"]

src/python/pants/engine/internals/scheduler.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@
5353
FallibleProcessResultWithPlatform,
5454
InteractiveProcess,
5555
InteractiveProcessResult,
56-
MultiPlatformProcess,
56+
Process,
5757
)
5858
from pants.engine.rules import Rule, RuleIndex, TaskRule
5959
from pants.engine.unions import UnionMembership, is_union
@@ -157,7 +157,7 @@ def __init__(
157157
digest_subset=DigestSubset,
158158
download_file=DownloadFile,
159159
platform=Platform,
160-
multi_platform_process=MultiPlatformProcess,
160+
process=Process,
161161
process_result=FallibleProcessResultWithPlatform,
162162
coroutine=CoroutineType,
163163
session_values=SessionValues,

src/python/pants/engine/process.py

+5-34
Original file line numberDiff line numberDiff line change
@@ -69,6 +69,7 @@ class Process:
6969
is_nailgunnable: bool
7070
execution_slot_variable: str | None
7171
cache_scope: ProcessCacheScope
72+
platform: str | None
7273

7374
def __init__(
7475
self,
@@ -87,6 +88,7 @@ def __init__(
8788
is_nailgunnable: bool = False,
8889
execution_slot_variable: str | None = None,
8990
cache_scope: ProcessCacheScope = ProcessCacheScope.SUCCESSFUL,
91+
platform: Platform | None = None,
9092
) -> None:
9193
"""Request to run a subprocess, similar to subprocess.Popen.
9294
@@ -131,32 +133,7 @@ def __init__(
131133
self.is_nailgunnable = is_nailgunnable
132134
self.execution_slot_variable = execution_slot_variable
133135
self.cache_scope = cache_scope
134-
135-
136-
@frozen_after_init
137-
@dataclass(unsafe_hash=True)
138-
class MultiPlatformProcess:
139-
platform_constraints: tuple[str | None, ...]
140-
processes: Tuple[Process, ...]
141-
142-
def __init__(self, request_dict: dict[Platform | None, Process]) -> None:
143-
if len(request_dict) == 0:
144-
raise ValueError("At least one platform-constrained Process must be passed.")
145-
serialized_constraints = tuple(
146-
constraint.value if constraint else None for constraint in request_dict
147-
)
148-
if len([req.description for req in request_dict.values()]) != 1:
149-
raise ValueError(
150-
f"The `description` of all processes in a {MultiPlatformProcess.__name__} must "
151-
f"be identical, but got: {list(request_dict.values())}."
152-
)
153-
154-
self.platform_constraints = serialized_constraints
155-
self.processes = tuple(request_dict.values())
156-
157-
@property
158-
def product_description(self) -> ProductDescription:
159-
return ProductDescription(self.processes[0].description)
136+
self.platform = platform.value if platform is not None else None
160137

161138

162139
@dataclass(frozen=True)
@@ -231,14 +208,8 @@ def __init__(
231208

232209

233210
@rule
234-
def get_multi_platform_request_description(req: MultiPlatformProcess) -> ProductDescription:
235-
return req.product_description
236-
237-
238-
@rule
239-
def upcast_process(req: Process) -> MultiPlatformProcess:
240-
"""This rule allows an Process to be run as a platform compatible MultiPlatformProcess."""
241-
return MultiPlatformProcess({None: req})
211+
def get_multi_platform_request_description(req: Process) -> ProductDescription:
212+
return ProductDescription(req.description)
242213

243214

244215
@rule

src/rust/engine/process_execution/src/cache.rs

+6-13
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,8 @@ use workunit_store::{
1616
};
1717

1818
use crate::{
19-
Context, FallibleProcessResultWithPlatform, MultiPlatformProcess, Platform, Process,
20-
ProcessCacheScope, ProcessMetadata, ProcessResultSource,
19+
Context, FallibleProcessResultWithPlatform, Platform, Process, ProcessCacheScope,
20+
ProcessMetadata, ProcessResultSource,
2121
};
2222

2323
#[allow(dead_code)]
@@ -53,22 +53,15 @@ impl CommandRunner {
5353

5454
#[async_trait]
5555
impl crate::CommandRunner for CommandRunner {
56-
fn extract_compatible_request(&self, req: &MultiPlatformProcess) -> Option<Process> {
57-
self.underlying.extract_compatible_request(req)
58-
}
59-
6056
async fn run(
6157
&self,
6258
context: Context,
6359
workunit: &mut RunningWorkunit,
64-
req: MultiPlatformProcess,
60+
req: Process,
6561
) -> Result<FallibleProcessResultWithPlatform, String> {
6662
let cache_lookup_start = Instant::now();
67-
let write_failures_to_cache = req
68-
.0
69-
.values()
70-
.any(|process| process.cache_scope == ProcessCacheScope::Always);
71-
let digest = crate::digest(req.clone(), &self.metadata);
63+
let write_failures_to_cache = req.cache_scope == ProcessCacheScope::Always;
64+
let digest = crate::digest(&req, &self.metadata);
7265
let key = digest.hash;
7366

7467
let context2 = context.clone();
@@ -77,7 +70,7 @@ impl crate::CommandRunner for CommandRunner {
7770
"local_cache_read".to_owned(),
7871
WorkunitMetadata {
7972
level: Level::Trace,
80-
desc: Some(format!("Local cache lookup: {}", req.user_facing_name())),
73+
desc: Some(format!("Local cache lookup: {}", req.description)),
8174
..WorkunitMetadata::default()
8275
},
8376
|workunit| async move {

src/rust/engine/process_execution/src/lib.rs

+15-91
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@
3131
extern crate derivative;
3232

3333
use std::collections::{BTreeMap, BTreeSet};
34-
use std::convert::TryFrom;
34+
use std::convert::{TryFrom, TryInto};
3535
use std::path::PathBuf;
3636
use std::sync::Arc;
3737

@@ -40,7 +40,7 @@ pub use log::Level;
4040
use async_semaphore::AsyncSemaphore;
4141
use async_trait::async_trait;
4242
use bazel_protos::gen::build::bazel::remote::execution::v2 as remexec;
43-
use hashing::{Digest, EMPTY_FINGERPRINT};
43+
use hashing::Digest;
4444
use remexec::ExecutedActionMetadata;
4545
use serde::{Deserialize, Serialize};
4646
use workunit_store::{RunningWorkunit, WorkunitStore};
@@ -323,51 +323,6 @@ impl Process {
323323
}
324324
}
325325

326-
impl TryFrom<MultiPlatformProcess> for Process {
327-
type Error = String;
328-
329-
fn try_from(req: MultiPlatformProcess) -> Result<Self, Self::Error> {
330-
match req.0.get(&None) {
331-
Some(crossplatform_req) => Ok(crossplatform_req.clone()),
332-
None => Err(String::from(
333-
"Cannot coerce to a simple Process, no cross platform request exists.",
334-
)),
335-
}
336-
}
337-
}
338-
339-
///
340-
/// A container of platform constrained processes.
341-
///
342-
#[derive(Derivative, Clone, Debug, Eq, PartialEq, Hash)]
343-
pub struct MultiPlatformProcess(pub BTreeMap<Option<Platform>, Process>);
344-
345-
impl MultiPlatformProcess {
346-
pub fn user_facing_name(&self) -> String {
347-
self
348-
.0
349-
.iter()
350-
.next()
351-
.map(|(_platforms, process)| process.description.clone())
352-
.unwrap_or_else(|| "<Unnamed process>".to_string())
353-
}
354-
355-
pub fn workunit_level(&self) -> log::Level {
356-
self
357-
.0
358-
.iter()
359-
.next()
360-
.map(|(_platforms, process)| process.level)
361-
.unwrap_or(Level::Info)
362-
}
363-
}
364-
365-
impl From<Process> for MultiPlatformProcess {
366-
fn from(proc: Process) -> Self {
367-
MultiPlatformProcess(vec![(None, proc)].into_iter().collect())
368-
}
369-
}
370-
371326
///
372327
/// Metadata surrounding an Process which factors into its cache key when cached
373328
/// externally from the engine graph (e.g. when using remote execution or an external process
@@ -526,40 +481,15 @@ pub trait CommandRunner: Send + Sync {
526481
&self,
527482
context: Context,
528483
workunit: &mut RunningWorkunit,
529-
req: MultiPlatformProcess,
484+
req: Process,
530485
) -> Result<FallibleProcessResultWithPlatform, String>;
531-
532-
///
533-
/// Given a multi platform request which may have some platform
534-
/// constraints determine if any of the requests contained within are compatible
535-
/// with the current command runners platform configuration. If so return the
536-
/// first candidate that will be run if the multi platform request is submitted to
537-
/// `fn run(..)`
538-
fn extract_compatible_request(&self, req: &MultiPlatformProcess) -> Option<Process>;
539486
}
540487

541488
// TODO(#8513) possibly move to the MEPR struct, or to the hashing crate?
542-
pub fn digest(req: MultiPlatformProcess, metadata: &ProcessMetadata) -> Digest {
543-
let mut hashes: Vec<String> = req
544-
.0
545-
.values()
546-
.map(|process| crate::remote::make_execute_request(process, metadata.clone()).unwrap())
547-
.map(|(_a, _b, er)| {
548-
er.action_digest
549-
.map(|d| d.hash)
550-
.unwrap_or_else(|| EMPTY_FINGERPRINT.to_hex())
551-
})
552-
.collect();
553-
hashes.sort();
554-
Digest::of_bytes(
555-
hashes
556-
.iter()
557-
.fold(String::new(), |mut acc, hash| {
558-
acc.push_str(hash);
559-
acc
560-
})
561-
.as_bytes(),
562-
)
489+
pub fn digest(process: &Process, metadata: &ProcessMetadata) -> Digest {
490+
let (_, _, execute_request) =
491+
crate::remote::make_execute_request(process, metadata.clone()).unwrap();
492+
execute_request.action_digest.unwrap().try_into().unwrap()
563493
}
564494

565495
///
@@ -584,7 +514,7 @@ impl CommandRunner for BoundedCommandRunner {
584514
&self,
585515
context: Context,
586516
workunit: &mut RunningWorkunit,
587-
mut req: MultiPlatformProcess,
517+
mut process: Process,
588518
) -> Result<FallibleProcessResultWithPlatform, String> {
589519
let semaphore = self.inner.1.clone();
590520
let inner = self.inner.clone();
@@ -593,28 +523,22 @@ impl CommandRunner for BoundedCommandRunner {
593523
.with_acquired(|concurrency_id| {
594524
log::debug!(
595525
"Running {} under semaphore with concurrency id: {}",
596-
req.user_facing_name(),
526+
process.description,
597527
concurrency_id
598528
);
599529
std::mem::drop(blocking_token);
600530

601-
for (_, process) in req.0.iter_mut() {
602-
if let Some(ref execution_slot_env_var) = process.execution_slot_variable {
603-
let execution_slot = format!("{}", concurrency_id);
604-
process
605-
.env
606-
.insert(execution_slot_env_var.clone(), execution_slot);
607-
}
531+
if let Some(ref execution_slot_env_var) = process.execution_slot_variable {
532+
let execution_slot = format!("{}", concurrency_id);
533+
process
534+
.env
535+
.insert(execution_slot_env_var.clone(), execution_slot);
608536
}
609537

610-
inner.0.run(context, workunit, req)
538+
inner.0.run(context, workunit, process)
611539
})
612540
.await
613541
}
614-
615-
fn extract_compatible_request(&self, req: &MultiPlatformProcess) -> Option<Process> {
616-
self.inner.0.extract_compatible_request(req)
617-
}
618542
}
619543

620544
impl From<Box<BoundedCommandRunner>> for Arc<dyn CommandRunner> {

src/rust/engine/process_execution/src/local.rs

+2-12
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ use tryfuture::try_future;
3232
use workunit_store::{in_workunit, Level, Metric, RunningWorkunit, WorkunitMetadata};
3333

3434
use crate::{
35-
Context, FallibleProcessResultWithPlatform, MultiPlatformProcess, NamedCaches, Platform, Process,
35+
Context, FallibleProcessResultWithPlatform, NamedCaches, Platform, Process,
3636
ProcessResultMetadata, ProcessResultSource,
3737
};
3838

@@ -233,25 +233,15 @@ impl ChildResults {
233233

234234
#[async_trait]
235235
impl super::CommandRunner for CommandRunner {
236-
fn extract_compatible_request(&self, req: &MultiPlatformProcess) -> Option<Process> {
237-
for compatible_constraint in vec![None, self.platform.into()].iter() {
238-
if let Some(compatible_req) = req.0.get(compatible_constraint) {
239-
return Some(compatible_req.clone());
240-
}
241-
}
242-
None
243-
}
244-
245236
///
246237
/// Runs a command on this machine in the passed working directory.
247238
///
248239
async fn run(
249240
&self,
250241
context: Context,
251242
_workunit: &mut RunningWorkunit,
252-
req: MultiPlatformProcess,
243+
req: Process,
253244
) -> Result<FallibleProcessResultWithPlatform, String> {
254-
let req = self.extract_compatible_request(&req).unwrap();
255245
let req_debug_repr = format!("{:#?}", req);
256246
in_workunit!(
257247
context.workunit_store.clone(),

0 commit comments

Comments
 (0)