Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[dagster-pipes-rust] - Add support for closing the pipes context #71

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions libraries/pipes/implementations/rust/CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@

### Added

- (pull/71) Added the `close` method to `PipesContext`
- (pull/67) Added the `data_version` parameter to `report_asset_materialization`
- (pull/60) Added `AssetCheckSeverity` to the jsonschema definitions
- (pull/59) Moved dagster pipes version into a constant
Expand Down
99 changes: 99 additions & 0 deletions libraries/pipes/implementations/rust/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ use serde_json::json;
use serde_json::Map;
use serde_json::Value;
use thiserror::Error;
use types::PipesException;

use crate::context_loader::{DefaultLoader as PipesDefaultContextLoader, PayloadErrorKind};
use crate::params_loader::{EnvVarLoader as PipesEnvVarParamsLoader, ParamsError};
Expand Down Expand Up @@ -60,6 +61,20 @@ where
})
}

pub fn close(&mut self, exc: Option<PipesException>) -> Result<(), MessageWriteError> {
let params = exc.map(|e| {
HashMap::from([
("cause", e.cause.map(|c| json!(c))),
("context", e.context.map(|c| json!(c))),
("message", e.message.map(|m| json!(m))),
("name", e.name.map(|n| json!(n))),
("stack", e.stack.map(|s| json!(s))),
])
});
let closed_message = PipesMessage::new(Method::Closed, params);
self.message_channel.write_message(closed_message)
}

pub fn report_asset_materialization(
&mut self,
asset_key: &str,
Expand Down Expand Up @@ -97,6 +112,15 @@ where
}
}

impl<W> Drop for PipesContext<W>
where
W: MessageWriter,
{
fn drop(&mut self) {
let _ = self.close(None);
}
}

#[derive(Debug, Error)]
#[non_exhaustive]
pub enum DagsterPipesError {
Expand Down Expand Up @@ -132,6 +156,7 @@ pub fn open_dagster_pipes() -> Result<PipesContext<PipesDefaultMessageWriter>, D

#[cfg(test)]
mod tests {
use rstest::rstest;
use std::collections::HashMap;
use std::fs;
use tempfile::NamedTempFile;
Expand Down Expand Up @@ -273,4 +298,78 @@ mod tests {
)
);
}

#[rstest]
#[case(
Some(PipesException {
cause: Box::new(None),
context: Box::new(None),
message: Some("error".to_string()),
name: Some("Error".to_string()),
stack: Some(vec!["line1".to_string(), "line2".to_string()]),
}),
json!({
"__dagster_pipes_version": "0.1",
"method": "closed",
"params": {
"cause": null,
"context": null,
"message": "error",
"name": "Error",
"stack": ["line1", "line2"]
},
})
)]
#[case(
None,
json!({
"__dagster_pipes_version": "0.1",
"method": "closed",
"params": null,
})
)]
fn test_close_pipes_context(
#[case] exc: Option<PipesException>,
#[case] expected_message: serde_json::Value,
) {
let file = NamedTempFile::new().unwrap();
let mut context: PipesContext<DefaultWriter> = PipesContext {
message_channel: DefaultChannel::File(FileChannel::new(file.path().into())),
data: PipesContextData {
asset_keys: Some(vec!["asset1".to_string()]),
run_id: "012345".to_string(),
..Default::default()
},
};
context.close(exc).expect("Failed to close context");
assert_eq!(
serde_json::from_str::<serde_json::Value>(&fs::read_to_string(file.path()).unwrap())
.unwrap(),
expected_message
);
}

#[test]
fn test_close_pipes_context_when_out_of_scope() {
let file = NamedTempFile::new().unwrap();
{
let _: PipesContext<DefaultWriter> = PipesContext {
message_channel: DefaultChannel::File(FileChannel::new(file.path().into())),
data: PipesContextData {
asset_keys: Some(vec!["asset1".to_string()]),
run_id: "012345".to_string(),
..Default::default()
},
};
}
assert_eq!(
serde_json::from_str::<serde_json::Value>(&fs::read_to_string(file.path()).unwrap())
.unwrap(),
json!({
"__dagster_pipes_version": "0.1",
"method": "closed",
"params": null,
})
);
}
}
Loading