-
Notifications
You must be signed in to change notification settings - Fork 1.5k
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
[runless] report_asset_materialization endpoint #16602
Merged
alangenfeld
merged 2 commits into
master
from
al/09-18-_runless_report_asset_materialization_endpoint
Oct 6, 2023
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -8,6 +8,11 @@ | |
from dagster import __version__ as dagster_version | ||
from dagster._annotations import deprecated | ||
from dagster._core.debug import DebugRunPayload | ||
from dagster._core.definitions.data_version import ( | ||
DATA_VERSION_IS_USER_PROVIDED_TAG, | ||
DATA_VERSION_TAG, | ||
) | ||
from dagster._core.definitions.events import AssetKey, AssetMaterialization | ||
from dagster._core.storage.cloud_storage_compute_log_manager import CloudStorageComputeLogManager | ||
from dagster._core.storage.compute_log_manager import ComputeIOType | ||
from dagster._core.storage.local_compute_log_manager import LocalComputeLogManager | ||
|
@@ -195,6 +200,107 @@ async def download_captured_logs_endpoint(self, request: Request): | |
filebase = "__".join(log_key) | ||
return FileResponse(location, filename=f"{filebase}.{file_extension}") | ||
|
||
async def report_asset_materialization_endpoint(self, request: Request) -> JSONResponse: | ||
# Record a runless asset materialization event. | ||
# The asset key is passed as url path with / delimiting parts or as a query param. | ||
# Properties can be passed as json post body or query params, with that order of precedence. | ||
|
||
context = self.make_request_context(request) | ||
|
||
body_content_type = request.headers.get("content-type") | ||
if body_content_type is None: | ||
json_body = {} | ||
elif body_content_type == "application/json": | ||
json_body = await request.json() | ||
else: | ||
return JSONResponse( | ||
{ | ||
"error": ( | ||
f"Unhandled content type {body_content_type}, expect no body or" | ||
" application/json" | ||
), | ||
}, | ||
status_code=400, | ||
) | ||
|
||
asset_key = None | ||
if request.path_params.get(ReportAssetMatParam.asset_key): | ||
# use from_user_string to treat / as multipart key separator | ||
asset_key = AssetKey.from_user_string(request.path_params["asset_key"]) | ||
elif ReportAssetMatParam.asset_key in json_body: | ||
asset_key = AssetKey(json_body[ReportAssetMatParam.asset_key]) | ||
elif ReportAssetMatParam.asset_key in request.query_params: | ||
asset_key = AssetKey.from_db_string(request.query_params["asset_key"]) | ||
|
||
if asset_key is None: | ||
return JSONResponse( | ||
{ | ||
"error": ( | ||
"Empty asset key, must provide asset key as url path after" | ||
" /report_asset_materialization/ or query param asset_key." | ||
), | ||
}, | ||
status_code=400, | ||
) | ||
|
||
tags = None | ||
if ReportAssetMatParam.data_version in json_body: | ||
tags = { | ||
DATA_VERSION_TAG: json_body[ReportAssetMatParam.data_version], | ||
DATA_VERSION_IS_USER_PROVIDED_TAG: "true", | ||
} | ||
elif ReportAssetMatParam.data_version in request.query_params: | ||
tags = { | ||
DATA_VERSION_TAG: request.query_params[ReportAssetMatParam.data_version], | ||
DATA_VERSION_IS_USER_PROVIDED_TAG: "true", | ||
} | ||
|
||
partition = None | ||
if ReportAssetMatParam.partition in json_body: | ||
partition = json_body[ReportAssetMatParam.partition] | ||
elif ReportAssetMatParam.partition in request.query_params: | ||
partition = request.query_params[ReportAssetMatParam.partition] | ||
|
||
description = None | ||
if ReportAssetMatParam.description in json_body: | ||
description = json_body[ReportAssetMatParam.description] | ||
elif ReportAssetMatParam.description in request.query_params: | ||
description = request.query_params[ReportAssetMatParam.description] | ||
|
||
metadata = None | ||
if ReportAssetMatParam.metadata in json_body: | ||
metadata = json_body["metadata"] | ||
elif ReportAssetMatParam.metadata in request.query_params: | ||
try: | ||
metadata = json.loads(request.query_params[ReportAssetMatParam.metadata]) | ||
except Exception as exc: | ||
return JSONResponse( | ||
{ | ||
"error": f"Error parsing metadata json: {exc}", | ||
}, | ||
status_code=400, | ||
) | ||
|
||
try: | ||
mat = AssetMaterialization( | ||
asset_key=asset_key, | ||
partition=partition, | ||
metadata=metadata, | ||
description=description, | ||
tags=tags, | ||
) | ||
except Exception as exc: | ||
return JSONResponse( | ||
{ | ||
"error": f"Error constructing AssetMaterialization: {exc}", | ||
}, | ||
status_code=400, | ||
) | ||
|
||
context.instance.report_runless_asset_event(mat) | ||
|
||
return JSONResponse({}) | ||
|
||
def index_html_endpoint(self, request: Request): | ||
"""Serves root html.""" | ||
index_path = self.relative_path("webapp/build/index.html") | ||
|
@@ -304,6 +410,11 @@ def build_routes(self): | |
"/download_debug/{run_id:str}", | ||
self.download_debug_file_endpoint, | ||
), | ||
Route( | ||
"/report_asset_materialization/{asset_key:path}", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. differs from |
||
self.report_asset_materialization_endpoint, | ||
methods=["POST"], | ||
), | ||
Route("/{path:path}", self.index_html_endpoint), | ||
Route("/", self.index_html_endpoint), | ||
] | ||
|
@@ -345,3 +456,17 @@ def send_wrapper(message: Message): | |
return send(message) | ||
|
||
await self.app(scope, receive, send_wrapper) | ||
|
||
|
||
class ReportAssetMatParam: | ||
"""Class to collect all supported args by report_asset_materialization endpoint | ||
to ensure consistency with related APIs. | ||
|
||
note: Enum not used to avoid value type problems X(str, Enum) doesn't work as partition conflicts with keyword | ||
""" | ||
|
||
asset_key = "asset_key" | ||
data_version = "data_version" | ||
metadata = "metadata" | ||
description = "description" | ||
partition = "partition" |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@smackesey to confirm this
data_version
handling