Skip to content

Commit

Permalink
internal: Create dep inference request type (pantsbuild#19001)
Browse files Browse the repository at this point in the history
Boilerplate to introduce a request type for native dependency inference
suggested here:
pantsbuild#18985 (comment)

Adds a "metadata" field to the CacheKey type. Is it naive to store the
string as is in the key? Afaict, the `PersistentCache` is addressed by
the hash of the bytes of the key, not the key bytes themselves -> Should
be fine.

---------

Co-authored-by: Joshua Cannon <[email protected]>
  • Loading branch information
tobni and thejcannon authored Jun 1, 2023
1 parent 0584653 commit 5becc13
Show file tree
Hide file tree
Showing 12 changed files with 256 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
from pants.engine.environment import EnvironmentName
from pants.engine.fs import CreateDigest, Digest, FileContent, MergeDigests
from pants.engine.internals.native_dep_inference import NativeParsedPythonDependencies
from pants.engine.internals.native_engine import NativeDependenciesRequest
from pants.engine.process import Process, ProcessResult
from pants.engine.rules import Get, MultiGet, collect_rules, rule
from pants.engine.unions import UnionMembership, UnionRule, union
Expand Down Expand Up @@ -194,7 +195,8 @@ async def parse_python_dependencies(
has_custom_dep_inferences = len(union_membership[PythonDependencyVisitorRequest]) > 1
if python_infer_subsystem.use_rust_parser and not has_custom_dep_inferences:
native_result = await Get(
NativeParsedPythonDependencies, Digest, stripped_sources.snapshot.digest
NativeParsedPythonDependencies,
NativeDependenciesRequest(stripped_sources.snapshot.digest),
)
imports = dict(native_result.imports)
assets = set()
Expand Down
1 change: 0 additions & 1 deletion src/python/pants/engine/fs.py
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,6 @@ def from_snapshots(cls, ours: Snapshot, theirs: Snapshot) -> "SnapshotDiff":


def rules():
# Keep in sync with `intrinsics.rs`.
return (
QueryRule(Digest, (CreateDigest,)),
QueryRule(Digest, (PathGlobs,)),
Expand Down
31 changes: 31 additions & 0 deletions src/python/pants/engine/internals/native_engine.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,37 @@ class PyStubCAS:
def remove(self, digest: FileDigest | Digest) -> bool: ...
def action_cache_len(self) -> int: ...

# ------------------------------------------------------------------------------
# Dependency inference
# ------------------------------------------------------------------------------

class InferenceMetadata:
@staticmethod
def javascript(
package_root: str, import_patterns: dict[str, list[str]]
) -> InferenceMetadata: ...
def __eq__(self, other: InferenceMetadata | Any) -> bool: ...
def __hash__(self) -> int: ...
def __repr__(self) -> str: ...

class NativeDependenciesRequest:
"""A request to parse the dependencies of a file.
* The `digest` is expected to contain exactly one source file.
* Depending on the implementation, a `metadata` structure
can be passed. It will be supplied to the native parser, and
it will be incorporated into the cache key.
Example:
result = await Get(NativeParsedPythonDependencies, NativeDependenciesRequest(input_digest, None)
"""

def __init__(self, digest: Digest, metadata: InferenceMetadata | None = None) -> None: ...
def __eq__(self, other: NativeDependenciesRequest | Any) -> bool: ...
def __hash__(self) -> int: ...
def __repr__(self) -> str: ...

# ------------------------------------------------------------------------------
# (etc.)
# ------------------------------------------------------------------------------
Expand Down
20 changes: 20 additions & 0 deletions src/python/pants/engine/internals/native_engine_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
# Copyright 2023 Pants project contributors (see CONTRIBUTORS.md).
# Licensed under the Apache License, Version 2.0 (see LICENSE).
from pants.engine.internals.native_engine import (
EMPTY_DIGEST,
InferenceMetadata,
NativeDependenciesRequest,
)


def test_can_construct_javascript_metadata() -> None:
InferenceMetadata.javascript(
package_root="some/dir", import_patterns={"a-pattern-*": ["replaces-me-*"]}
)


def test_can_construct_native_dependencies_request() -> None:
NativeDependenciesRequest(EMPTY_DIGEST, None)
NativeDependenciesRequest(
EMPTY_DIGEST, InferenceMetadata.javascript(package_root="some/dir", import_patterns={})
)
17 changes: 17 additions & 0 deletions src/rust/engine/protos/protos/pants/cache.proto
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,23 @@ message CacheKey {
build.bazel.remote.execution.v2.Digest digest = 2;
}

message DependencyInferenceRequest {
build.bazel.remote.execution.v2.Digest input_file_digest = 1;
oneof metadata {
JavascriptInferenceMetadata js = 2;
}
}


message JavascriptInferenceMetadata {
message ImportPattern {
string pattern = 1;
repeated string replacements = 2;
}
string package_root = 1;
repeated ImportPattern import_patterns = 2;
}

// A URL and Digest tuple, which is itself digested and used as a CacheKey. ObservedURLs
// collectively represent the set of digests that we have ever observed for a particular URL:
// their cache value is always empty.
Expand Down
24 changes: 24 additions & 0 deletions src/rust/engine/protos/src/hashing.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
// Copyright 2023 Pants project contributors (see CONTRIBUTORS.md).
// Licensed under the Apache License, Version 2.0 (see LICENSE).
use std::hash::{Hash, Hasher};

use crate::gen::pants::cache::dependency_inference_request::Metadata;
use crate::gen::pants::cache::JavascriptInferenceMetadata;

impl Hash for JavascriptInferenceMetadata {
fn hash<H: Hasher>(&self, state: &mut H) {
self.package_root.hash(state);
for pattern in &self.import_patterns {
pattern.pattern.hash(state);
pattern.replacements.hash(state);
}
}
}

impl Hash for Metadata {
fn hash<H: Hasher>(&self, state: &mut H) {
match self {
Metadata::Js(m) => m.hash(state),
}
}
}
2 changes: 2 additions & 0 deletions src/rust/engine/protos/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -65,3 +65,5 @@ mod verification;
pub use crate::verification::verify_directory_canonical;
#[cfg(test)]
mod verification_tests;

mod hashing;
107 changes: 107 additions & 0 deletions src/rust/engine/src/externs/dep_inference.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
// Copyright 2023 Pants project contributors (see CONTRIBUTORS.md).
// Licensed under the Apache License, Version 2.0 (see LICENSE).
use std::collections::hash_map::DefaultHasher;
use std::hash::{Hash, Hasher};

use pyo3::basic::CompareOp;
use pyo3::prelude::*;
use pyo3::types::PyDict;
use pyo3::{IntoPy, PyObject, Python};

use fs::DirectoryDigest;
use protos::gen::pants::cache::{
dependency_inference_request, javascript_inference_metadata, JavascriptInferenceMetadata,
};

use crate::externs::fs::PyDigest;

pub(crate) fn register(m: &PyModule) -> PyResult<()> {
m.add_class::<PyNativeDependenciesRequest>()?;
m.add_class::<PyInferenceMetadata>()
}

#[pyclass(name = "InferenceMetadata")]
#[derive(Clone, Debug, PartialEq)]
pub struct PyInferenceMetadata(pub dependency_inference_request::Metadata);

#[pymethods]
impl PyInferenceMetadata {
#[staticmethod]
fn javascript(package_root: String, import_patterns: &PyDict) -> PyResult<Self> {
use javascript_inference_metadata::ImportPattern;
let import_patterns: PyResult<Vec<ImportPattern>> = import_patterns
.iter()
.map(|(key, value)| {
Ok(ImportPattern {
pattern: key.extract()?,
replacements: value.extract()?,
})
})
.collect();
Ok(Self(dependency_inference_request::Metadata::Js(
JavascriptInferenceMetadata {
package_root,
import_patterns: import_patterns?,
},
)))
}

fn __richcmp__(&self, other: &Self, op: CompareOp, py: Python) -> PyObject {
match op {
CompareOp::Eq => (self == other).into_py(py),
CompareOp::Ne => (self != other).into_py(py),
_ => py.NotImplemented(),
}
}

fn __repr__(&self) -> String {
format!("InferenceMetadata({:?})", self.0)
}

fn __hash__(&self) -> u64 {
let mut s = DefaultHasher::new();
self.0.hash(&mut s);
s.finish()
}
}

#[pyclass(name = "NativeDependenciesRequest")]
#[derive(Clone, Debug, PartialEq)]
pub struct PyNativeDependenciesRequest {
pub directory_digest: DirectoryDigest,
pub metadata: Option<dependency_inference_request::Metadata>,
}

#[pymethods]
impl PyNativeDependenciesRequest {
#[new]
fn __new__(digest: PyDigest, metadata: Option<PyInferenceMetadata>) -> Self {
Self {
directory_digest: digest.0,
metadata: metadata.map(|inner| inner.0),
}
}

fn __hash__(&self) -> u64 {
let mut s = DefaultHasher::new();
self.directory_digest.hash(&mut s);
self.metadata.hash(&mut s);
s.finish()
}

fn __repr__(&self) -> String {
format!(
"NativeDependenciesRequest('{}', {:?})",
PyDigest(self.directory_digest.clone()),
self.metadata
)
}

fn __richcmp__(&self, other: &Self, op: CompareOp, py: Python) -> PyObject {
match op {
CompareOp::Eq => (self == other).into_py(py),
CompareOp::Ne => (self != other).into_py(py),
_ => py.NotImplemented(),
}
}
}
4 changes: 4 additions & 0 deletions src/rust/engine/src/externs/interface.rs
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ fn native_engine(py: Python, m: &PyModule) -> PyO3Result<()> {
externs::scheduler::register(m)?;
externs::testutil::register(m)?;
externs::workunits::register(m)?;
externs::dep_inference::register(m)?;

m.add("PollTimeout", py.get_type::<PollTimeout>())?;

Expand Down Expand Up @@ -234,6 +235,9 @@ impl PyTypes {
docker_resolve_image_request: TypeId::new(docker_resolve_image_request),
docker_resolve_image_result: TypeId::new(docker_resolve_image_result),
parsed_python_deps_result: TypeId::new(parsed_python_deps_result),
deps_request: TypeId::new(
py.get_type::<externs::dep_inference::PyNativeDependenciesRequest>(),
),
})))
}
}
Expand Down
1 change: 1 addition & 0 deletions src/rust/engine/src/externs/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use crate::interning::Interns;
use crate::python::{Failure, Key, TypeId, Value};

mod address;
pub mod dep_inference;
pub mod engine_aware;
pub mod fs;
mod interface;
Expand Down
63 changes: 46 additions & 17 deletions src/rust/engine/src/intrinsics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ use crate::tasks::Intrinsic;
use crate::types::Types;
use crate::Failure;
use dep_inference::python::get_dependencies;
use protos::gen::pants::cache::{CacheKey, CacheKeyType};
use protos::gen::pants::cache::{CacheKey, CacheKeyType, DependencyInferenceRequest};

use bytes::Bytes;
use futures::future::{BoxFuture, FutureExt, TryFutureExt};
Expand All @@ -39,18 +39,20 @@ use process_execution::local::{
use process_execution::{ManagedChild, Platform, ProcessExecutionStrategy};
use rule_graph::DependencyKey;
use stdio::TryCloneAsFile;
use store::{SnapshotOps, SubsetParams};
use store::{SnapshotOps, Store, SubsetParams};

use crate::externs::dep_inference::PyNativeDependenciesRequest;
use workunit_store::{in_workunit, Level};

use grpc_util::prost::MessageExt;

type IntrinsicFn =
Box<dyn Fn(Context, Vec<Value>) -> BoxFuture<'static, NodeResult<Value>> + Send + Sync>;

pub struct Intrinsics {
intrinsics: IndexMap<Intrinsic, IntrinsicFn>,
}

// NB: Keep in sync with `rules()` in `src/python/pants/engine/fs.py`.
impl Intrinsics {
pub fn new(types: &Types) -> Intrinsics {
let mut intrinsics: IndexMap<Intrinsic, IntrinsicFn> = IndexMap::new();
Expand Down Expand Up @@ -142,7 +144,7 @@ impl Intrinsics {
intrinsics.insert(
Intrinsic {
product: types.parsed_python_deps_result,
inputs: vec![DependencyKey::new(types.directory_digest)],
inputs: vec![DependencyKey::new(types.deps_request)],
},
Box::new(parse_python_deps),
);
Expand Down Expand Up @@ -750,14 +752,18 @@ fn docker_resolve_image(
.boxed()
}

fn parse_python_deps(context: Context, args: Vec<Value>) -> BoxFuture<'static, NodeResult<Value>> {
async move {
let core = &context.core;
let store = core.store();
let directory_digest = Python::with_gil(|py| {
let py_digest = (*args[0]).as_ref(py);
lift_directory_digest(py_digest)
})?;
struct PreparedInferenceRequest {
pub path: PathBuf,
pub digest: Digest,
inner: DependencyInferenceRequest,
}

impl PreparedInferenceRequest {
pub async fn prepare(args: Vec<Value>, store: &Store) -> NodeResult<Self> {
let PyNativeDependenciesRequest {
directory_digest,
metadata,
} = Python::with_gil(|py| (*args[0]).as_ref(py).extract())?;

let mut path = None;
let mut digest = None;
Expand All @@ -777,23 +783,46 @@ fn parse_python_deps(context: Context, args: Vec<Value>) -> BoxFuture<'static, N
}
let path = path.unwrap();
let digest = digest.unwrap();
Ok(Self {
path,
digest,
inner: DependencyInferenceRequest {
input_file_digest: Some(digest.into()),
metadata,
},
})
}

fn cache_key(&self) -> CacheKey {
CacheKey {
key_type: CacheKeyType::DepInferenceRequest.into(),
digest: Some(Digest::of_bytes(&self.inner.to_bytes()).into()),
}
}
}

fn parse_python_deps(context: Context, args: Vec<Value>) -> BoxFuture<'static, NodeResult<Value>> {
async move {
let core = &context.core;
let store = core.store();
let prepared_inference_request = PreparedInferenceRequest::prepare(args, &store).await?;
in_workunit!(
"parse_python_dependencies",
Level::Debug,
desc = Some(format!("Determine Python dependencies for {path:?}")),
desc = Some(format!(
"Determine Python dependencies for {:?}",
&prepared_inference_request.path
)),
|_workunit| async move {
let cache_key = CacheKey {
key_type: CacheKeyType::DepInferenceRequest.into(),
digest: Some(digest.into()),
};
let cache_key = prepared_inference_request.cache_key();
let cached_result = core.local_cache.load(&cache_key).await?;

let result = if let Some(result) =
cached_result.and_then(|bytes| serde_json::from_slice(&bytes).ok())
{
result
} else {
let PreparedInferenceRequest { digest, path, .. } = prepared_inference_request;
let bytes = store
.load_file_bytes_with(digest, |bytes| Vec::from(bytes))
.await?;
Expand Down
1 change: 1 addition & 0 deletions src/rust/engine/src/types.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,4 +35,5 @@ pub struct Types {
pub docker_resolve_image_request: TypeId,
pub docker_resolve_image_result: TypeId,
pub parsed_python_deps_result: TypeId,
pub deps_request: TypeId,
}

0 comments on commit 5becc13

Please sign in to comment.