diff --git a/examples/airline_demo/airline_demo_tests/test_types.py b/examples/airline_demo/airline_demo_tests/test_types.py --- a/examples/airline_demo/airline_demo_tests/test_types.py +++ b/examples/airline_demo/airline_demo_tests/test_types.py @@ -137,23 +137,25 @@ def test_spark_dataframe_output_csv(): spark = SparkSession.builder.getOrCreate() - num_df = ( - spark.read.format("csv") - .options(header="true", inferSchema="true") - .load(file_relative_path(__file__, "num.csv")) - ) - - assert num_df.collect() == [Row(num1=1, num2=2)] - @solid - def emit(_): + @solid( + input_defs=[InputDefinition("num_df", DataFrame)], + output_defs=[OutputDefinition(DataFrame)], + required_resource_keys={"pyspark"}, + ) + def emit(_, num_df): + assert num_df.collect() == [Row(num1=1, num2=2)] return num_df - @solid(input_defs=[InputDefinition("df", DataFrame)], output_defs=[OutputDefinition(DataFrame)]) + @solid( + input_defs=[InputDefinition("df", DataFrame)], + output_defs=[OutputDefinition(DataFrame)], + required_resource_keys={"pyspark"}, + ) def passthrough_df(_context, df): return df - @pipeline + @pipeline(mode_defs=[ModeDefinition(resource_defs={"pyspark": pyspark_resource})]) def passthrough(): passthrough_df(emit()) @@ -163,16 +165,25 @@ passthrough, run_config={ "solids": { + "emit": { + "inputs": { + "num_df": { + "csv": { + "path": file_relative_path(__file__, "num.csv"), + "header": True, + "inferSchema": True, + } + } + } + }, "passthrough_df": { - "outputs": [{"result": {"csv": {"path": file_name, "header": True}}}] - } + "outputs": [{"result": {"csv": {"path": file_name, "header": True}}}], + }, }, }, ) - from_file_df = ( - spark.read.format("csv").options(header="true", inferSchema="true").load(file_name) - ) + from_file_df = spark.read.format("csv").options(header="true").load(file_name) assert ( result.result_for_solid("passthrough_df").output_value().collect() diff --git a/js_modules/dagit/src/assets/types/AssetQuery.ts b/js_modules/dagit/src/assets/types/AssetQuery.ts --- a/js_modules/dagit/src/assets/types/AssetQuery.ts +++ b/js_modules/dagit/src/assets/types/AssetQuery.ts @@ -25,6 +25,13 @@ path: string; } +export interface AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -68,7 +75,7 @@ value: number; } -export type AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries = AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventPathMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventTextMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries = AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventPathMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventTextMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface AssetQuery_assetOrError_Asset_graphMaterializations_materializationEvent_materialization { __typename: "Materialization"; @@ -106,6 +113,13 @@ path: string; } +export interface AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -149,7 +163,7 @@ value: number; } -export type AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries = AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventPathMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventTextMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries = AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventPathMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventTextMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface AssetQuery_assetOrError_Asset_lastMaterializations_materializationEvent_materialization { __typename: "Materialization"; diff --git a/js_modules/dagit/src/filteredSchema.generated.json b/js_modules/dagit/src/filteredSchema.generated.json --- a/js_modules/dagit/src/filteredSchema.generated.json +++ b/js_modules/dagit/src/filteredSchema.generated.json @@ -1 +1 @@ -{"__schema":{"types":[{"name":"Query","kind":"OBJECT","possibleTypes":null},{"name":"String","kind":"SCALAR","possibleTypes":null},{"name":"RepositoriesOrError","kind":"UNION","possibleTypes":[{"name":"RepositoryConnection"},{"name":"PythonError"}]},{"name":"RepositoryConnection","kind":"OBJECT","possibleTypes":null},{"name":"Repository","kind":"OBJECT","possibleTypes":null},{"name":"ID","kind":"SCALAR","possibleTypes":null},{"name":"RepositoryLocation","kind":"OBJECT","possibleTypes":null},{"name":"Boolean","kind":"SCALAR","possibleTypes":null},{"name":"Pipeline","kind":"OBJECT","possibleTypes":null},{"name":"SolidContainer","kind":"INTERFACE","possibleTypes":[{"name":"Pipeline"},{"name":"CompositeSolidDefinition"}]},{"name":"Solid","kind":"OBJECT","possibleTypes":null},{"name":"ISolidDefinition","kind":"INTERFACE","possibleTypes":[{"name":"SolidDefinition"},{"name":"CompositeSolidDefinition"}]},{"name":"MetadataItemDefinition","kind":"OBJECT","possibleTypes":null},{"name":"InputDefinition","kind":"OBJECT","possibleTypes":null},{"name":"SolidDefinition","kind":"OBJECT","possibleTypes":null},{"name":"OutputDefinition","kind":"OBJECT","possibleTypes":null},{"name":"DagsterType","kind":"INTERFACE","possibleTypes":[{"name":"RegularDagsterType"},{"name":"ListDagsterType"},{"name":"NullableDagsterType"}]},{"name":"ConfigType","kind":"INTERFACE","possibleTypes":[{"name":"RegularConfigType"},{"name":"ArrayConfigType"},{"name":"ScalarUnionConfigType"},{"name":"NullableConfigType"},{"name":"EnumConfigType"},{"name":"CompositeConfigType"}]},{"name":"ResourceRequirement","kind":"OBJECT","possibleTypes":null},{"name":"ConfigTypeField","kind":"OBJECT","possibleTypes":null},{"name":"Input","kind":"OBJECT","possibleTypes":null},{"name":"Output","kind":"OBJECT","possibleTypes":null},{"name":"IPipelineSnapshot","kind":"INTERFACE","possibleTypes":[{"name":"Pipeline"},{"name":"PipelineSnapshot"}]},{"name":"DagsterTypeOrError","kind":"UNION","possibleTypes":[{"name":"RegularDagsterType"},{"name":"PipelineNotFoundError"},{"name":"DagsterTypeNotFoundError"},{"name":"PythonError"}]},{"name":"RegularDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"PipelineNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"Error","kind":"INTERFACE","possibleTypes":[{"name":"PipelineNotFoundError"},{"name":"DagsterTypeNotFoundError"},{"name":"PythonError"},{"name":"PipelineRunNotFoundError"},{"name":"RepositoryNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PipelineSnapshotNotFoundError"},{"name":"SchedulerNotDefinedError"},{"name":"ScheduleDefinitionNotFoundError"},{"name":"PartitionSetNotFoundError"},{"name":"InvalidPipelineRunsFilterError"},{"name":"RunGroupNotFoundError"},{"name":"ModeNotFoundError"},{"name":"AssetsNotSupportedError"},{"name":"PipelineRunConflict"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"},{"name":"ReloadNotSupported"},{"name":"RepositoryLocationNotFound"},{"name":"ScheduleNotFoundError"},{"name":"ConfigTypeNotFoundError"},{"name":"ScheduleStateNotFoundError"}]},{"name":"DagsterTypeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PythonError","kind":"OBJECT","possibleTypes":null},{"name":"Mode","kind":"OBJECT","possibleTypes":null},{"name":"Resource","kind":"OBJECT","possibleTypes":null},{"name":"Logger","kind":"OBJECT","possibleTypes":null},{"name":"SolidHandle","kind":"OBJECT","possibleTypes":null},{"name":"PipelineTag","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRun","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunStatus","kind":"ENUM","possibleTypes":null},{"name":"PipelineReference","kind":"INTERFACE","possibleTypes":[{"name":"PipelineSnapshot"},{"name":"UnknownPipeline"}]},{"name":"PipelineRunStatsOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRunStatsSnapshot"},{"name":"PythonError"}]},{"name":"PipelineRunStatsSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"Int","kind":"SCALAR","possibleTypes":null},{"name":"Float","kind":"SCALAR","possibleTypes":null},{"name":"PipelineRunStepStats","kind":"OBJECT","possibleTypes":null},{"name":"StepEventStatus","kind":"ENUM","possibleTypes":null},{"name":"Materialization","kind":"OBJECT","possibleTypes":null},{"name":"DisplayableEvent","kind":"INTERFACE","possibleTypes":[{"name":"Materialization"},{"name":"ExpectationResult"},{"name":"FailureMetadata"},{"name":"TypeCheck"},{"name":"ObjectStoreOperationResult"},{"name":"EngineEvent"}]},{"name":"EventMetadataEntry","kind":"INTERFACE","possibleTypes":[{"name":"EventPathMetadataEntry"},{"name":"EventJsonMetadataEntry"},{"name":"EventTextMetadataEntry"},{"name":"EventUrlMetadataEntry"},{"name":"EventMarkdownMetadataEntry"},{"name":"EventPythonArtifactMetadataEntry"},{"name":"EventFloatMetadataEntry"}]},{"name":"AssetKey","kind":"OBJECT","possibleTypes":null},{"name":"ExpectationResult","kind":"OBJECT","possibleTypes":null},{"name":"ComputeLogs","kind":"OBJECT","possibleTypes":null},{"name":"ComputeLogFile","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionPlan","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStep","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepInput","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepOutput","kind":"OBJECT","possibleTypes":null},{"name":"StepKind","kind":"ENUM","possibleTypes":null},{"name":"Asset","kind":"OBJECT","possibleTypes":null},{"name":"AssetMaterialization","kind":"OBJECT","possibleTypes":null},{"name":"StepMaterializationEvent","kind":"OBJECT","possibleTypes":null},{"name":"MessageEvent","kind":"INTERFACE","possibleTypes":[{"name":"StepMaterializationEvent"},{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"LogMessageEvent"},{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"LogLevel","kind":"ENUM","possibleTypes":null},{"name":"StepEvent","kind":"INTERFACE","possibleTypes":[{"name":"StepMaterializationEvent"},{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"PipelineRunOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRun"},{"name":"PipelineRunNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineRunNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleDefinition","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleState","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStatus","kind":"ENUM","possibleTypes":null},{"name":"ScheduleTick","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickStatus","kind":"ENUM","possibleTypes":null},{"name":"ScheduleTickSpecificData","kind":"UNION","possibleTypes":[{"name":"ScheduleTickSuccessData"},{"name":"ScheduleTickFailureData"}]},{"name":"ScheduleTickSuccessData","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickFailureData","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickStatsSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryOrigin","kind":"UNION","possibleTypes":[{"name":"PythonRepositoryOrigin"},{"name":"GrpcRepositoryOrigin"}]},{"name":"PythonRepositoryOrigin","kind":"OBJECT","possibleTypes":null},{"name":"CodePointer","kind":"OBJECT","possibleTypes":null},{"name":"CodePointerMetadata","kind":"OBJECT","possibleTypes":null},{"name":"GrpcRepositoryOrigin","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleRunConfigOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleRunConfig"},{"name":"PythonError"}]},{"name":"ScheduleRunConfig","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSet","kind":"OBJECT","possibleTypes":null},{"name":"PartitionsOrError","kind":"UNION","possibleTypes":[{"name":"Partitions"},{"name":"PythonError"}]},{"name":"Partitions","kind":"OBJECT","possibleTypes":null},{"name":"Partition","kind":"OBJECT","possibleTypes":null},{"name":"PartitionRunConfigOrError","kind":"UNION","possibleTypes":[{"name":"PartitionRunConfig"},{"name":"PythonError"}]},{"name":"PartitionRunConfig","kind":"OBJECT","possibleTypes":null},{"name":"PartitionTagsOrError","kind":"UNION","possibleTypes":[{"name":"PartitionTags"},{"name":"PythonError"}]},{"name":"PartitionTags","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunsFilter","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutionTag","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelinePreset","kind":"OBJECT","possibleTypes":null},{"name":"UsedSolid","kind":"OBJECT","possibleTypes":null},{"name":"SolidInvocationSite","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryOrError","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"Repository"},{"name":"RepositoryNotFoundError"}]},{"name":"RepositoryNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RepositorySelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelineOrError","kind":"UNION","possibleTypes":[{"name":"Pipeline"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PythonError"}]},{"name":"InvalidSubsetError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelineSnapshotOrError","kind":"UNION","possibleTypes":[{"name":"PipelineSnapshot"},{"name":"PipelineSnapshotNotFoundError"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSnapshotNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"SchedulerOrError","kind":"UNION","possibleTypes":[{"name":"Scheduler"},{"name":"SchedulerNotDefinedError"},{"name":"PythonError"}]},{"name":"Scheduler","kind":"OBJECT","possibleTypes":null},{"name":"SchedulerNotDefinedError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleDefinitionOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleDefinition"},{"name":"ScheduleDefinitionNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleDefinitionNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ScheduleDefinitionsOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleDefinitions"},{"name":"RepositoryNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleDefinitions","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStatesOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleStates"},{"name":"RepositoryNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleStates","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSetsOrError","kind":"UNION","possibleTypes":[{"name":"PartitionSets"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PartitionSets","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSetOrError","kind":"UNION","possibleTypes":[{"name":"PartitionSet"},{"name":"PartitionSetNotFoundError"},{"name":"PythonError"}]},{"name":"PartitionSetNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunsOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRuns"},{"name":"InvalidPipelineRunsFilterError"},{"name":"PythonError"}]},{"name":"PipelineRuns","kind":"OBJECT","possibleTypes":null},{"name":"InvalidPipelineRunsFilterError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineTagAndValues","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupOrError","kind":"UNION","possibleTypes":[{"name":"RunGroup"},{"name":"RunGroupNotFoundError"},{"name":"PythonError"}]},{"name":"RunGroup","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupsOrError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationResult","kind":"UNION","possibleTypes":[{"name":"InvalidSubsetError"},{"name":"PipelineConfigValidationValid"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineConfigValidationValid","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationInvalid","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationError","kind":"INTERFACE","possibleTypes":[{"name":"RuntimeMismatchConfigError"},{"name":"MissingFieldConfigError"},{"name":"MissingFieldsConfigError"},{"name":"FieldNotDefinedConfigError"},{"name":"FieldsNotDefinedConfigError"},{"name":"SelectorTypeConfigError"}]},{"name":"EvaluationStack","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationStackEntry","kind":"UNION","possibleTypes":[{"name":"EvaluationStackListItemEntry"},{"name":"EvaluationStackPathEntry"}]},{"name":"EvaluationStackListItemEntry","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationStackPathEntry","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationErrorReason","kind":"ENUM","possibleTypes":null},{"name":"RunConfigData","kind":"SCALAR","possibleTypes":null},{"name":"ExecutionPlanOrError","kind":"UNION","possibleTypes":[{"name":"ExecutionPlan"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PythonError"}]},{"name":"RunConfigSchemaOrError","kind":"UNION","possibleTypes":[{"name":"RunConfigSchema"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"ModeNotFoundError"},{"name":"PythonError"}]},{"name":"RunConfigSchema","kind":"OBJECT","possibleTypes":null},{"name":"ModeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"Instance","kind":"OBJECT","possibleTypes":null},{"name":"RunLauncher","kind":"OBJECT","possibleTypes":null},{"name":"AssetsOrError","kind":"UNION","possibleTypes":[{"name":"AssetConnection"},{"name":"AssetsNotSupportedError"}]},{"name":"AssetConnection","kind":"OBJECT","possibleTypes":null},{"name":"AssetsNotSupportedError","kind":"OBJECT","possibleTypes":null},{"name":"AssetOrError","kind":"UNION","possibleTypes":[{"name":"Asset"},{"name":"AssetsNotSupportedError"}]},{"name":"AssetKeyInput","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"Mutation","kind":"OBJECT","possibleTypes":null},{"name":"LaunchPipelineExecutionResult","kind":"UNION","possibleTypes":[{"name":"LaunchPipelineRunSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"LaunchPipelineRunSuccess","kind":"OBJECT","possibleTypes":null},{"name":"InvalidStepError","kind":"OBJECT","possibleTypes":null},{"name":"InvalidOutputError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunConflict","kind":"OBJECT","possibleTypes":null},{"name":"PresetNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ConflictingExecutionParamsError","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionParams","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutionMetadata","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"LaunchPipelineReexecutionResult","kind":"UNION","possibleTypes":[{"name":"LaunchPipelineRunSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"ReconcileSchedulerStateMutationResult","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"ReconcileSchedulerStateSuccess"}]},{"name":"ReconcileSchedulerStateSuccess","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleMutationResult","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"ScheduleStateResult"}]},{"name":"ScheduleStateResult","kind":"OBJECT","possibleTypes":null},{"name":"TerminatePipelineExecutionResult","kind":"UNION","possibleTypes":[{"name":"TerminatePipelineExecutionSuccess"},{"name":"TerminatePipelineExecutionFailure"},{"name":"PipelineRunNotFoundError"},{"name":"PythonError"}]},{"name":"TerminatePipelineExecutionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TerminatePipelineExecutionFailure","kind":"OBJECT","possibleTypes":null},{"name":"DeletePipelineRunResult","kind":"UNION","possibleTypes":[{"name":"DeletePipelineRunSuccess"},{"name":"PythonError"},{"name":"PipelineRunNotFoundError"}]},{"name":"DeletePipelineRunSuccess","kind":"OBJECT","possibleTypes":null},{"name":"ReloadRepositoryLocationMutationResult","kind":"UNION","possibleTypes":[{"name":"RepositoryLocation"},{"name":"ReloadNotSupported"},{"name":"RepositoryLocationNotFound"}]},{"name":"ReloadNotSupported","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryLocationNotFound","kind":"OBJECT","possibleTypes":null},{"name":"PartitionBackfillResult","kind":"UNION","possibleTypes":[{"name":"PartitionBackfillSuccess"},{"name":"PartitionSetNotFoundError"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"PartitionBackfillSuccess","kind":"OBJECT","possibleTypes":null},{"name":"PartitionBackfillParams","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PartitionSetSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutePlanResult","kind":"UNION","possibleTypes":[{"name":"ExecutePlanSuccess"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"InvalidStepError"},{"name":"PythonError"}]},{"name":"ExecutePlanSuccess","kind":"OBJECT","possibleTypes":null},{"name":"Retries","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"RetriesPreviousAttempts","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecuteRunInProcessResult","kind":"UNION","possibleTypes":[{"name":"ExecuteRunInProcessSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"},{"name":"PipelineRunNotFoundError"}]},{"name":"ExecuteRunInProcessSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TriggerMutationResult","kind":"UNION","possibleTypes":[{"name":"TriggerExecutionSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"TriggerExecutionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TriggerSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"Subscription","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunLogsSubscriptionPayload","kind":"UNION","possibleTypes":[{"name":"PipelineRunLogsSubscriptionSuccess"},{"name":"PipelineRunLogsSubscriptionFailure"}]},{"name":"PipelineRunLogsSubscriptionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunEvent","kind":"UNION","possibleTypes":[{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"LogMessageEvent"},{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"StepMaterializationEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"ExecutionStepFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"FailureMetadata","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepInputEvent","kind":"OBJECT","possibleTypes":null},{"name":"TypeCheck","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepOutputEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepSkippedEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepStartEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepSuccessEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepUpForRetryEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepRestartEvent","kind":"OBJECT","possibleTypes":null},{"name":"LogMessageEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineEvent","kind":"INTERFACE","possibleTypes":[{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"}]},{"name":"PipelineInitFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineStartEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSuccessEvent","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationEvent","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationResult","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationType","kind":"ENUM","possibleTypes":null},{"name":"StepExpectationResultEvent","kind":"OBJECT","possibleTypes":null},{"name":"EngineEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookCompletedEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookSkippedEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookErroredEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunLogsSubscriptionFailure","kind":"OBJECT","possibleTypes":null},{"name":"Cursor","kind":"SCALAR","possibleTypes":null},{"name":"ComputeIOType","kind":"ENUM","possibleTypes":null},{"name":"__Schema","kind":"OBJECT","possibleTypes":null},{"name":"__Type","kind":"OBJECT","possibleTypes":null},{"name":"__TypeKind","kind":"ENUM","possibleTypes":null},{"name":"__Field","kind":"OBJECT","possibleTypes":null},{"name":"__InputValue","kind":"OBJECT","possibleTypes":null},{"name":"__EnumValue","kind":"OBJECT","possibleTypes":null},{"name":"__Directive","kind":"OBJECT","possibleTypes":null},{"name":"__DirectiveLocation","kind":"ENUM","possibleTypes":null},{"name":"RegularConfigType","kind":"OBJECT","possibleTypes":null},{"name":"ArrayConfigType","kind":"OBJECT","possibleTypes":null},{"name":"WrappingConfigType","kind":"INTERFACE","possibleTypes":[{"name":"ArrayConfigType"},{"name":"NullableConfigType"}]},{"name":"ScalarUnionConfigType","kind":"OBJECT","possibleTypes":null},{"name":"NullableConfigType","kind":"OBJECT","possibleTypes":null},{"name":"EnumConfigType","kind":"OBJECT","possibleTypes":null},{"name":"EnumConfigValue","kind":"OBJECT","possibleTypes":null},{"name":"CompositeConfigType","kind":"OBJECT","possibleTypes":null},{"name":"ListDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"WrappingDagsterType","kind":"INTERFACE","possibleTypes":[{"name":"ListDagsterType"},{"name":"NullableDagsterType"}]},{"name":"NullableDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"CompositeSolidDefinition","kind":"OBJECT","possibleTypes":null},{"name":"InputMapping","kind":"OBJECT","possibleTypes":null},{"name":"OutputMapping","kind":"OBJECT","possibleTypes":null},{"name":"UnknownPipeline","kind":"OBJECT","possibleTypes":null},{"name":"MissingRunIdErrorEvent","kind":"OBJECT","possibleTypes":null},{"name":"EventPathMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventJsonMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventTextMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventUrlMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventMarkdownMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventPythonArtifactMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventFloatMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RuntimeMismatchConfigError","kind":"OBJECT","possibleTypes":null},{"name":"MissingFieldConfigError","kind":"OBJECT","possibleTypes":null},{"name":"MissingFieldsConfigError","kind":"OBJECT","possibleTypes":null},{"name":"FieldNotDefinedConfigError","kind":"OBJECT","possibleTypes":null},{"name":"FieldsNotDefinedConfigError","kind":"OBJECT","possibleTypes":null},{"name":"SelectorTypeConfigError","kind":"OBJECT","possibleTypes":null},{"name":"ConfigTypeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStateNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PageInfo","kind":"OBJECT","possibleTypes":null}]}} \ No newline at end of file +{"__schema":{"types":[{"name":"Query","kind":"OBJECT","possibleTypes":null},{"name":"String","kind":"SCALAR","possibleTypes":null},{"name":"RepositoriesOrError","kind":"UNION","possibleTypes":[{"name":"RepositoryConnection"},{"name":"PythonError"}]},{"name":"RepositoryConnection","kind":"OBJECT","possibleTypes":null},{"name":"Repository","kind":"OBJECT","possibleTypes":null},{"name":"ID","kind":"SCALAR","possibleTypes":null},{"name":"RepositoryLocation","kind":"OBJECT","possibleTypes":null},{"name":"Boolean","kind":"SCALAR","possibleTypes":null},{"name":"Pipeline","kind":"OBJECT","possibleTypes":null},{"name":"SolidContainer","kind":"INTERFACE","possibleTypes":[{"name":"Pipeline"},{"name":"CompositeSolidDefinition"}]},{"name":"Solid","kind":"OBJECT","possibleTypes":null},{"name":"ISolidDefinition","kind":"INTERFACE","possibleTypes":[{"name":"SolidDefinition"},{"name":"CompositeSolidDefinition"}]},{"name":"MetadataItemDefinition","kind":"OBJECT","possibleTypes":null},{"name":"InputDefinition","kind":"OBJECT","possibleTypes":null},{"name":"SolidDefinition","kind":"OBJECT","possibleTypes":null},{"name":"OutputDefinition","kind":"OBJECT","possibleTypes":null},{"name":"DagsterType","kind":"INTERFACE","possibleTypes":[{"name":"RegularDagsterType"},{"name":"ListDagsterType"},{"name":"NullableDagsterType"}]},{"name":"ConfigType","kind":"INTERFACE","possibleTypes":[{"name":"RegularConfigType"},{"name":"ArrayConfigType"},{"name":"ScalarUnionConfigType"},{"name":"NullableConfigType"},{"name":"EnumConfigType"},{"name":"CompositeConfigType"}]},{"name":"ResourceRequirement","kind":"OBJECT","possibleTypes":null},{"name":"ConfigTypeField","kind":"OBJECT","possibleTypes":null},{"name":"Input","kind":"OBJECT","possibleTypes":null},{"name":"Output","kind":"OBJECT","possibleTypes":null},{"name":"IPipelineSnapshot","kind":"INTERFACE","possibleTypes":[{"name":"Pipeline"},{"name":"PipelineSnapshot"}]},{"name":"DagsterTypeOrError","kind":"UNION","possibleTypes":[{"name":"RegularDagsterType"},{"name":"PipelineNotFoundError"},{"name":"DagsterTypeNotFoundError"},{"name":"PythonError"}]},{"name":"RegularDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"PipelineNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"Error","kind":"INTERFACE","possibleTypes":[{"name":"PipelineNotFoundError"},{"name":"DagsterTypeNotFoundError"},{"name":"PythonError"},{"name":"PipelineRunNotFoundError"},{"name":"RepositoryNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PipelineSnapshotNotFoundError"},{"name":"SchedulerNotDefinedError"},{"name":"ScheduleDefinitionNotFoundError"},{"name":"PartitionSetNotFoundError"},{"name":"InvalidPipelineRunsFilterError"},{"name":"RunGroupNotFoundError"},{"name":"ModeNotFoundError"},{"name":"AssetsNotSupportedError"},{"name":"PipelineRunConflict"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"},{"name":"ReloadNotSupported"},{"name":"RepositoryLocationNotFound"},{"name":"ScheduleNotFoundError"},{"name":"ConfigTypeNotFoundError"},{"name":"ScheduleStateNotFoundError"}]},{"name":"DagsterTypeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PythonError","kind":"OBJECT","possibleTypes":null},{"name":"Mode","kind":"OBJECT","possibleTypes":null},{"name":"Resource","kind":"OBJECT","possibleTypes":null},{"name":"Logger","kind":"OBJECT","possibleTypes":null},{"name":"SolidHandle","kind":"OBJECT","possibleTypes":null},{"name":"PipelineTag","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRun","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunStatus","kind":"ENUM","possibleTypes":null},{"name":"PipelineReference","kind":"INTERFACE","possibleTypes":[{"name":"PipelineSnapshot"},{"name":"UnknownPipeline"}]},{"name":"PipelineRunStatsOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRunStatsSnapshot"},{"name":"PythonError"}]},{"name":"PipelineRunStatsSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"Int","kind":"SCALAR","possibleTypes":null},{"name":"Float","kind":"SCALAR","possibleTypes":null},{"name":"PipelineRunStepStats","kind":"OBJECT","possibleTypes":null},{"name":"StepEventStatus","kind":"ENUM","possibleTypes":null},{"name":"Materialization","kind":"OBJECT","possibleTypes":null},{"name":"DisplayableEvent","kind":"INTERFACE","possibleTypes":[{"name":"Materialization"},{"name":"ExpectationResult"},{"name":"FailureMetadata"},{"name":"TypeCheck"},{"name":"ObjectStoreOperationResult"},{"name":"EngineEvent"}]},{"name":"EventMetadataEntry","kind":"INTERFACE","possibleTypes":[{"name":"EventPathMetadataEntry"},{"name":"EventAddressMetadataEntry"},{"name":"EventJsonMetadataEntry"},{"name":"EventTextMetadataEntry"},{"name":"EventUrlMetadataEntry"},{"name":"EventMarkdownMetadataEntry"},{"name":"EventPythonArtifactMetadataEntry"},{"name":"EventFloatMetadataEntry"}]},{"name":"AssetKey","kind":"OBJECT","possibleTypes":null},{"name":"ExpectationResult","kind":"OBJECT","possibleTypes":null},{"name":"ComputeLogs","kind":"OBJECT","possibleTypes":null},{"name":"ComputeLogFile","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionPlan","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStep","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepInput","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepOutput","kind":"OBJECT","possibleTypes":null},{"name":"StepKind","kind":"ENUM","possibleTypes":null},{"name":"Asset","kind":"OBJECT","possibleTypes":null},{"name":"AssetMaterialization","kind":"OBJECT","possibleTypes":null},{"name":"StepMaterializationEvent","kind":"OBJECT","possibleTypes":null},{"name":"MessageEvent","kind":"INTERFACE","possibleTypes":[{"name":"StepMaterializationEvent"},{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"LogMessageEvent"},{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"LogLevel","kind":"ENUM","possibleTypes":null},{"name":"StepEvent","kind":"INTERFACE","possibleTypes":[{"name":"StepMaterializationEvent"},{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"PipelineRunOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRun"},{"name":"PipelineRunNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineRunNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleDefinition","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleState","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStatus","kind":"ENUM","possibleTypes":null},{"name":"ScheduleTick","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickStatus","kind":"ENUM","possibleTypes":null},{"name":"ScheduleTickSpecificData","kind":"UNION","possibleTypes":[{"name":"ScheduleTickSuccessData"},{"name":"ScheduleTickFailureData"}]},{"name":"ScheduleTickSuccessData","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickFailureData","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleTickStatsSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryOrigin","kind":"UNION","possibleTypes":[{"name":"PythonRepositoryOrigin"},{"name":"GrpcRepositoryOrigin"}]},{"name":"PythonRepositoryOrigin","kind":"OBJECT","possibleTypes":null},{"name":"CodePointer","kind":"OBJECT","possibleTypes":null},{"name":"CodePointerMetadata","kind":"OBJECT","possibleTypes":null},{"name":"GrpcRepositoryOrigin","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleRunConfigOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleRunConfig"},{"name":"PythonError"}]},{"name":"ScheduleRunConfig","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSet","kind":"OBJECT","possibleTypes":null},{"name":"PartitionsOrError","kind":"UNION","possibleTypes":[{"name":"Partitions"},{"name":"PythonError"}]},{"name":"Partitions","kind":"OBJECT","possibleTypes":null},{"name":"Partition","kind":"OBJECT","possibleTypes":null},{"name":"PartitionRunConfigOrError","kind":"UNION","possibleTypes":[{"name":"PartitionRunConfig"},{"name":"PythonError"}]},{"name":"PartitionRunConfig","kind":"OBJECT","possibleTypes":null},{"name":"PartitionTagsOrError","kind":"UNION","possibleTypes":[{"name":"PartitionTags"},{"name":"PythonError"}]},{"name":"PartitionTags","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunsFilter","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutionTag","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelinePreset","kind":"OBJECT","possibleTypes":null},{"name":"UsedSolid","kind":"OBJECT","possibleTypes":null},{"name":"SolidInvocationSite","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryOrError","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"Repository"},{"name":"RepositoryNotFoundError"}]},{"name":"RepositoryNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RepositorySelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelineOrError","kind":"UNION","possibleTypes":[{"name":"Pipeline"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PythonError"}]},{"name":"InvalidSubsetError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PipelineSnapshotOrError","kind":"UNION","possibleTypes":[{"name":"PipelineSnapshot"},{"name":"PipelineSnapshotNotFoundError"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineSnapshot","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSnapshotNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"SchedulerOrError","kind":"UNION","possibleTypes":[{"name":"Scheduler"},{"name":"SchedulerNotDefinedError"},{"name":"PythonError"}]},{"name":"Scheduler","kind":"OBJECT","possibleTypes":null},{"name":"SchedulerNotDefinedError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleDefinitionOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleDefinition"},{"name":"ScheduleDefinitionNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleDefinitionNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ScheduleDefinitionsOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleDefinitions"},{"name":"RepositoryNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleDefinitions","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStatesOrError","kind":"UNION","possibleTypes":[{"name":"ScheduleStates"},{"name":"RepositoryNotFoundError"},{"name":"PythonError"}]},{"name":"ScheduleStates","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSetsOrError","kind":"UNION","possibleTypes":[{"name":"PartitionSets"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PartitionSets","kind":"OBJECT","possibleTypes":null},{"name":"PartitionSetOrError","kind":"UNION","possibleTypes":[{"name":"PartitionSet"},{"name":"PartitionSetNotFoundError"},{"name":"PythonError"}]},{"name":"PartitionSetNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunsOrError","kind":"UNION","possibleTypes":[{"name":"PipelineRuns"},{"name":"InvalidPipelineRunsFilterError"},{"name":"PythonError"}]},{"name":"PipelineRuns","kind":"OBJECT","possibleTypes":null},{"name":"InvalidPipelineRunsFilterError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineTagAndValues","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupOrError","kind":"UNION","possibleTypes":[{"name":"RunGroup"},{"name":"RunGroupNotFoundError"},{"name":"PythonError"}]},{"name":"RunGroup","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RunGroupsOrError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationResult","kind":"UNION","possibleTypes":[{"name":"InvalidSubsetError"},{"name":"PipelineConfigValidationValid"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PythonError"}]},{"name":"PipelineConfigValidationValid","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationInvalid","kind":"OBJECT","possibleTypes":null},{"name":"PipelineConfigValidationError","kind":"INTERFACE","possibleTypes":[{"name":"RuntimeMismatchConfigError"},{"name":"MissingFieldConfigError"},{"name":"MissingFieldsConfigError"},{"name":"FieldNotDefinedConfigError"},{"name":"FieldsNotDefinedConfigError"},{"name":"SelectorTypeConfigError"}]},{"name":"EvaluationStack","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationStackEntry","kind":"UNION","possibleTypes":[{"name":"EvaluationStackListItemEntry"},{"name":"EvaluationStackPathEntry"}]},{"name":"EvaluationStackListItemEntry","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationStackPathEntry","kind":"OBJECT","possibleTypes":null},{"name":"EvaluationErrorReason","kind":"ENUM","possibleTypes":null},{"name":"RunConfigData","kind":"SCALAR","possibleTypes":null},{"name":"ExecutionPlanOrError","kind":"UNION","possibleTypes":[{"name":"ExecutionPlan"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"PythonError"}]},{"name":"RunConfigSchemaOrError","kind":"UNION","possibleTypes":[{"name":"RunConfigSchema"},{"name":"PipelineNotFoundError"},{"name":"InvalidSubsetError"},{"name":"ModeNotFoundError"},{"name":"PythonError"}]},{"name":"RunConfigSchema","kind":"OBJECT","possibleTypes":null},{"name":"ModeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"Instance","kind":"OBJECT","possibleTypes":null},{"name":"RunLauncher","kind":"OBJECT","possibleTypes":null},{"name":"AssetsOrError","kind":"UNION","possibleTypes":[{"name":"AssetConnection"},{"name":"AssetsNotSupportedError"}]},{"name":"AssetConnection","kind":"OBJECT","possibleTypes":null},{"name":"AssetsNotSupportedError","kind":"OBJECT","possibleTypes":null},{"name":"AssetOrError","kind":"UNION","possibleTypes":[{"name":"Asset"},{"name":"AssetsNotSupportedError"}]},{"name":"AssetKeyInput","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"Mutation","kind":"OBJECT","possibleTypes":null},{"name":"LaunchPipelineExecutionResult","kind":"UNION","possibleTypes":[{"name":"LaunchPipelineRunSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"LaunchPipelineRunSuccess","kind":"OBJECT","possibleTypes":null},{"name":"InvalidStepError","kind":"OBJECT","possibleTypes":null},{"name":"InvalidOutputError","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunConflict","kind":"OBJECT","possibleTypes":null},{"name":"PresetNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ConflictingExecutionParamsError","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionParams","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutionMetadata","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"LaunchPipelineReexecutionResult","kind":"UNION","possibleTypes":[{"name":"LaunchPipelineRunSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"ReconcileSchedulerStateMutationResult","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"ReconcileSchedulerStateSuccess"}]},{"name":"ReconcileSchedulerStateSuccess","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleMutationResult","kind":"UNION","possibleTypes":[{"name":"PythonError"},{"name":"ScheduleStateResult"}]},{"name":"ScheduleStateResult","kind":"OBJECT","possibleTypes":null},{"name":"TerminatePipelineExecutionResult","kind":"UNION","possibleTypes":[{"name":"TerminatePipelineExecutionSuccess"},{"name":"TerminatePipelineExecutionFailure"},{"name":"PipelineRunNotFoundError"},{"name":"PythonError"}]},{"name":"TerminatePipelineExecutionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TerminatePipelineExecutionFailure","kind":"OBJECT","possibleTypes":null},{"name":"DeletePipelineRunResult","kind":"UNION","possibleTypes":[{"name":"DeletePipelineRunSuccess"},{"name":"PythonError"},{"name":"PipelineRunNotFoundError"}]},{"name":"DeletePipelineRunSuccess","kind":"OBJECT","possibleTypes":null},{"name":"ReloadRepositoryLocationMutationResult","kind":"UNION","possibleTypes":[{"name":"RepositoryLocation"},{"name":"ReloadNotSupported"},{"name":"RepositoryLocationNotFound"}]},{"name":"ReloadNotSupported","kind":"OBJECT","possibleTypes":null},{"name":"RepositoryLocationNotFound","kind":"OBJECT","possibleTypes":null},{"name":"PartitionBackfillResult","kind":"UNION","possibleTypes":[{"name":"PartitionBackfillSuccess"},{"name":"PartitionSetNotFoundError"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"PartitionBackfillSuccess","kind":"OBJECT","possibleTypes":null},{"name":"PartitionBackfillParams","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"PartitionSetSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecutePlanResult","kind":"UNION","possibleTypes":[{"name":"ExecutePlanSuccess"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"InvalidStepError"},{"name":"PythonError"}]},{"name":"ExecutePlanSuccess","kind":"OBJECT","possibleTypes":null},{"name":"Retries","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"RetriesPreviousAttempts","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"ExecuteRunInProcessResult","kind":"UNION","possibleTypes":[{"name":"ExecuteRunInProcessSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"},{"name":"PipelineRunNotFoundError"}]},{"name":"ExecuteRunInProcessSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TriggerMutationResult","kind":"UNION","possibleTypes":[{"name":"TriggerExecutionSuccess"},{"name":"InvalidStepError"},{"name":"InvalidOutputError"},{"name":"PipelineConfigValidationInvalid"},{"name":"PipelineNotFoundError"},{"name":"PipelineRunConflict"},{"name":"PythonError"},{"name":"PresetNotFoundError"},{"name":"ConflictingExecutionParamsError"}]},{"name":"TriggerExecutionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"TriggerSelector","kind":"INPUT_OBJECT","possibleTypes":null},{"name":"Subscription","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunLogsSubscriptionPayload","kind":"UNION","possibleTypes":[{"name":"PipelineRunLogsSubscriptionSuccess"},{"name":"PipelineRunLogsSubscriptionFailure"}]},{"name":"PipelineRunLogsSubscriptionSuccess","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunEvent","kind":"UNION","possibleTypes":[{"name":"ExecutionStepFailureEvent"},{"name":"ExecutionStepInputEvent"},{"name":"ExecutionStepOutputEvent"},{"name":"ExecutionStepSkippedEvent"},{"name":"ExecutionStepStartEvent"},{"name":"ExecutionStepSuccessEvent"},{"name":"ExecutionStepUpForRetryEvent"},{"name":"ExecutionStepRestartEvent"},{"name":"LogMessageEvent"},{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"},{"name":"ObjectStoreOperationEvent"},{"name":"StepExpectationResultEvent"},{"name":"StepMaterializationEvent"},{"name":"EngineEvent"},{"name":"HookCompletedEvent"},{"name":"HookSkippedEvent"},{"name":"HookErroredEvent"}]},{"name":"ExecutionStepFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"FailureMetadata","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepInputEvent","kind":"OBJECT","possibleTypes":null},{"name":"TypeCheck","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepOutputEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepSkippedEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepStartEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepSuccessEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepUpForRetryEvent","kind":"OBJECT","possibleTypes":null},{"name":"ExecutionStepRestartEvent","kind":"OBJECT","possibleTypes":null},{"name":"LogMessageEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineEvent","kind":"INTERFACE","possibleTypes":[{"name":"PipelineFailureEvent"},{"name":"PipelineInitFailureEvent"},{"name":"PipelineStartEvent"},{"name":"PipelineSuccessEvent"}]},{"name":"PipelineInitFailureEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineStartEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineSuccessEvent","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationEvent","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationResult","kind":"OBJECT","possibleTypes":null},{"name":"ObjectStoreOperationType","kind":"ENUM","possibleTypes":null},{"name":"StepExpectationResultEvent","kind":"OBJECT","possibleTypes":null},{"name":"EngineEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookCompletedEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookSkippedEvent","kind":"OBJECT","possibleTypes":null},{"name":"HookErroredEvent","kind":"OBJECT","possibleTypes":null},{"name":"PipelineRunLogsSubscriptionFailure","kind":"OBJECT","possibleTypes":null},{"name":"Cursor","kind":"SCALAR","possibleTypes":null},{"name":"ComputeIOType","kind":"ENUM","possibleTypes":null},{"name":"__Schema","kind":"OBJECT","possibleTypes":null},{"name":"__Type","kind":"OBJECT","possibleTypes":null},{"name":"__TypeKind","kind":"ENUM","possibleTypes":null},{"name":"__Field","kind":"OBJECT","possibleTypes":null},{"name":"__InputValue","kind":"OBJECT","possibleTypes":null},{"name":"__EnumValue","kind":"OBJECT","possibleTypes":null},{"name":"__Directive","kind":"OBJECT","possibleTypes":null},{"name":"__DirectiveLocation","kind":"ENUM","possibleTypes":null},{"name":"RegularConfigType","kind":"OBJECT","possibleTypes":null},{"name":"ArrayConfigType","kind":"OBJECT","possibleTypes":null},{"name":"WrappingConfigType","kind":"INTERFACE","possibleTypes":[{"name":"ArrayConfigType"},{"name":"NullableConfigType"}]},{"name":"ScalarUnionConfigType","kind":"OBJECT","possibleTypes":null},{"name":"NullableConfigType","kind":"OBJECT","possibleTypes":null},{"name":"EnumConfigType","kind":"OBJECT","possibleTypes":null},{"name":"EnumConfigValue","kind":"OBJECT","possibleTypes":null},{"name":"CompositeConfigType","kind":"OBJECT","possibleTypes":null},{"name":"ListDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"WrappingDagsterType","kind":"INTERFACE","possibleTypes":[{"name":"ListDagsterType"},{"name":"NullableDagsterType"}]},{"name":"NullableDagsterType","kind":"OBJECT","possibleTypes":null},{"name":"CompositeSolidDefinition","kind":"OBJECT","possibleTypes":null},{"name":"InputMapping","kind":"OBJECT","possibleTypes":null},{"name":"OutputMapping","kind":"OBJECT","possibleTypes":null},{"name":"UnknownPipeline","kind":"OBJECT","possibleTypes":null},{"name":"MissingRunIdErrorEvent","kind":"OBJECT","possibleTypes":null},{"name":"EventPathMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventAddressMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventJsonMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventTextMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventUrlMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventMarkdownMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventPythonArtifactMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"EventFloatMetadataEntry","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"RuntimeMismatchConfigError","kind":"OBJECT","possibleTypes":null},{"name":"MissingFieldConfigError","kind":"OBJECT","possibleTypes":null},{"name":"MissingFieldsConfigError","kind":"OBJECT","possibleTypes":null},{"name":"FieldNotDefinedConfigError","kind":"OBJECT","possibleTypes":null},{"name":"FieldsNotDefinedConfigError","kind":"OBJECT","possibleTypes":null},{"name":"SelectorTypeConfigError","kind":"OBJECT","possibleTypes":null},{"name":"ConfigTypeNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"ScheduleStateNotFoundError","kind":"OBJECT","possibleTypes":null},{"name":"PageInfo","kind":"OBJECT","possibleTypes":null}]}} \ No newline at end of file diff --git a/js_modules/dagit/src/runs/MetadataEntry.tsx b/js_modules/dagit/src/runs/MetadataEntry.tsx --- a/js_modules/dagit/src/runs/MetadataEntry.tsx +++ b/js_modules/dagit/src/runs/MetadataEntry.tsx @@ -63,6 +63,9 @@ ... on EventPathMetadataEntry { path } + ... on EventAddressMetadataEntry { + address + } ... on EventJsonMetadataEntry { jsonString } @@ -107,6 +110,8 @@ /> ); + case 'EventAddressMetadataEntry': + return entry.address; case 'EventJsonMetadataEntry': return ( diff --git a/js_modules/dagit/src/runs/types/LogsRowStructuredFragment.ts b/js_modules/dagit/src/runs/types/LogsRowStructuredFragment.ts --- a/js_modules/dagit/src/runs/types/LogsRowStructuredFragment.ts +++ b/js_modules/dagit/src/runs/types/LogsRowStructuredFragment.ts @@ -29,6 +29,13 @@ path: string; } +export interface LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -72,7 +79,7 @@ value: number; } -export type LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries = LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries = LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_StepMaterializationEvent_materialization { __typename: "Materialization"; @@ -155,6 +162,13 @@ path: string; } +export interface LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -198,7 +212,7 @@ value: number; } -export type LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_ExecutionStepFailureEvent_failureMetadata { __typename: "FailureMetadata"; @@ -222,6 +236,13 @@ path: string; } +export interface LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -265,7 +286,7 @@ value: number; } -export type LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_ExecutionStepInputEvent_typeCheck { __typename: "TypeCheck"; @@ -292,6 +313,13 @@ path: string; } +export interface LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -335,7 +363,7 @@ value: number; } -export type LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_ExecutionStepOutputEvent_typeCheck { __typename: "TypeCheck"; @@ -362,6 +390,13 @@ path: string; } +export interface LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -405,7 +440,7 @@ value: number; } -export type LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries = LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries = LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_StepExpectationResultEvent_expectationResult { __typename: "ExpectationResult"; @@ -431,6 +466,13 @@ path: string; } +export interface LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -474,7 +516,7 @@ value: number; } -export type LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_ObjectStoreOperationEvent_operationResult { __typename: "ObjectStoreOperationResult"; @@ -498,6 +540,13 @@ path: string; } +export interface LogsRowStructuredFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsRowStructuredFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -541,7 +590,7 @@ value: number; } -export type LogsRowStructuredFragment_EngineEvent_metadataEntries = LogsRowStructuredFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; +export type LogsRowStructuredFragment_EngineEvent_metadataEntries = LogsRowStructuredFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | LogsRowStructuredFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; export interface LogsRowStructuredFragment_EngineEvent_engineError_cause { __typename: "PythonError"; diff --git a/js_modules/dagit/src/runs/types/LogsScrollingTableMessageFragment.ts b/js_modules/dagit/src/runs/types/LogsScrollingTableMessageFragment.ts --- a/js_modules/dagit/src/runs/types/LogsScrollingTableMessageFragment.ts +++ b/js_modules/dagit/src/runs/types/LogsScrollingTableMessageFragment.ts @@ -29,6 +29,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -72,7 +79,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries = LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries = LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_StepMaterializationEvent_materialization { __typename: "Materialization"; @@ -155,6 +162,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -198,7 +212,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_ExecutionStepFailureEvent_failureMetadata { __typename: "FailureMetadata"; @@ -222,6 +236,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -265,7 +286,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_ExecutionStepInputEvent_typeCheck { __typename: "TypeCheck"; @@ -292,6 +313,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -335,7 +363,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_ExecutionStepOutputEvent_typeCheck { __typename: "TypeCheck"; @@ -362,6 +390,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -405,7 +440,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries = LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries = LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_StepExpectationResultEvent_expectationResult { __typename: "ExpectationResult"; @@ -431,6 +466,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -474,7 +516,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_ObjectStoreOperationEvent_operationResult { __typename: "ObjectStoreOperationResult"; @@ -498,6 +540,13 @@ path: string; } +export interface LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -541,7 +590,7 @@ value: number; } -export type LogsScrollingTableMessageFragment_EngineEvent_metadataEntries = LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; +export type LogsScrollingTableMessageFragment_EngineEvent_metadataEntries = LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | LogsScrollingTableMessageFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; export interface LogsScrollingTableMessageFragment_EngineEvent_engineError_cause { __typename: "PythonError"; diff --git a/js_modules/dagit/src/runs/types/MetadataEntryFragment.ts b/js_modules/dagit/src/runs/types/MetadataEntryFragment.ts --- a/js_modules/dagit/src/runs/types/MetadataEntryFragment.ts +++ b/js_modules/dagit/src/runs/types/MetadataEntryFragment.ts @@ -14,6 +14,13 @@ path: string; } +export interface MetadataEntryFragment_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface MetadataEntryFragment_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -57,4 +64,4 @@ value: number; } -export type MetadataEntryFragment = MetadataEntryFragment_EventPathMetadataEntry | MetadataEntryFragment_EventJsonMetadataEntry | MetadataEntryFragment_EventUrlMetadataEntry | MetadataEntryFragment_EventTextMetadataEntry | MetadataEntryFragment_EventMarkdownMetadataEntry | MetadataEntryFragment_EventPythonArtifactMetadataEntry | MetadataEntryFragment_EventFloatMetadataEntry; +export type MetadataEntryFragment = MetadataEntryFragment_EventPathMetadataEntry | MetadataEntryFragment_EventAddressMetadataEntry | MetadataEntryFragment_EventJsonMetadataEntry | MetadataEntryFragment_EventUrlMetadataEntry | MetadataEntryFragment_EventTextMetadataEntry | MetadataEntryFragment_EventMarkdownMetadataEntry | MetadataEntryFragment_EventPythonArtifactMetadataEntry | MetadataEntryFragment_EventFloatMetadataEntry; diff --git a/js_modules/dagit/src/runs/types/PipelineRunLogsSubscription.ts b/js_modules/dagit/src/runs/types/PipelineRunLogsSubscription.ts --- a/js_modules/dagit/src/runs/types/PipelineRunLogsSubscription.ts +++ b/js_modules/dagit/src/runs/types/PipelineRunLogsSubscription.ts @@ -30,6 +30,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -73,7 +80,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepMaterializationEvent_materialization { __typename: "Materialization"; @@ -159,6 +166,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -202,7 +216,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepFailureEvent_failureMetadata { __typename: "FailureMetadata"; @@ -227,6 +241,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -270,7 +291,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepInputEvent_typeCheck { __typename: "TypeCheck"; @@ -298,6 +319,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -341,7 +369,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ExecutionStepOutputEvent_typeCheck { __typename: "TypeCheck"; @@ -369,6 +397,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -412,7 +447,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_StepExpectationResultEvent_expectationResult { __typename: "ExpectationResult"; @@ -439,6 +474,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -482,7 +524,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_ObjectStoreOperationEvent_operationResult { __typename: "ObjectStoreOperationResult"; @@ -507,6 +549,13 @@ path: string; } +export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -550,7 +599,7 @@ value: number; } -export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventFloatMetadataEntry; +export type PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries = PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventPathMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventAddressMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventJsonMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventUrlMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventTextMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_metadataEntries_EventFloatMetadataEntry; export interface PipelineRunLogsSubscription_pipelineRunLogs_PipelineRunLogsSubscriptionSuccess_messages_EngineEvent_engineError_cause { __typename: "PythonError"; diff --git a/js_modules/dagit/src/runs/types/RunPipelineRunEventFragment.ts b/js_modules/dagit/src/runs/types/RunPipelineRunEventFragment.ts --- a/js_modules/dagit/src/runs/types/RunPipelineRunEventFragment.ts +++ b/js_modules/dagit/src/runs/types/RunPipelineRunEventFragment.ts @@ -29,6 +29,13 @@ path: string; } +export interface RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -72,7 +79,7 @@ value: number; } -export type RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries = RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries = RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_StepMaterializationEvent_materialization { __typename: "Materialization"; @@ -155,6 +162,13 @@ path: string; } +export interface RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -198,7 +212,7 @@ value: number; } -export type RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries = RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_ExecutionStepFailureEvent_failureMetadata { __typename: "FailureMetadata"; @@ -222,6 +236,13 @@ path: string; } +export interface RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -265,7 +286,7 @@ value: number; } -export type RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries = RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_ExecutionStepInputEvent_typeCheck { __typename: "TypeCheck"; @@ -292,6 +313,13 @@ path: string; } +export interface RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -335,7 +363,7 @@ value: number; } -export type RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries = RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_ExecutionStepOutputEvent_typeCheck { __typename: "TypeCheck"; @@ -362,6 +390,13 @@ path: string; } +export interface RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -405,7 +440,7 @@ value: number; } -export type RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries = RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries = RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_StepExpectationResultEvent_expectationResult { __typename: "ExpectationResult"; @@ -431,6 +466,13 @@ path: string; } +export interface RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -474,7 +516,7 @@ value: number; } -export type RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_ObjectStoreOperationEvent_operationResult { __typename: "ObjectStoreOperationResult"; @@ -498,6 +540,13 @@ path: string; } +export interface RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -541,7 +590,7 @@ value: number; } -export type RunPipelineRunEventFragment_EngineEvent_metadataEntries = RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; +export type RunPipelineRunEventFragment_EngineEvent_metadataEntries = RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventPathMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventAddressMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventJsonMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventUrlMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventTextMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventMarkdownMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventPythonArtifactMetadataEntry | RunPipelineRunEventFragment_EngineEvent_metadataEntries_EventFloatMetadataEntry; export interface RunPipelineRunEventFragment_EngineEvent_engineError_cause { __typename: "PythonError"; diff --git a/js_modules/dagit/src/schema.graphql b/js_modules/dagit/src/schema.graphql --- a/js_modules/dagit/src/schema.graphql +++ b/js_modules/dagit/src/schema.graphql @@ -208,6 +208,12 @@ fieldName: String! } +type EventAddressMetadataEntry implements EventMetadataEntry { + label: String! + description: String + address: String +} + type EventFloatMetadataEntry implements EventMetadataEntry { label: String! description: String @@ -682,7 +688,9 @@ enum ObjectStoreOperationType { SET_OBJECT + SET_EXTERNAL_OBJECT GET_OBJECT + GET_EXTERNAL_OBJECT RM_OBJECT CP_OBJECT } diff --git a/js_modules/dagit/src/types/RunMetadataProviderMessageFragment.ts b/js_modules/dagit/src/types/RunMetadataProviderMessageFragment.ts --- a/js_modules/dagit/src/types/RunMetadataProviderMessageFragment.ts +++ b/js_modules/dagit/src/types/RunMetadataProviderMessageFragment.ts @@ -25,8 +25,8 @@ markerEnd: string | null; } -export interface RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry { - __typename: "EventFloatMetadataEntry"; +export interface RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry" | "EventFloatMetadataEntry"; label: string; description: string | null; } @@ -74,7 +74,7 @@ name: string; } -export type RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries = RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventFloatMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry; +export type RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries = RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventAddressMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization_metadataEntries_EventPythonArtifactMetadataEntry; export interface RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization { __typename: "Materialization"; @@ -91,8 +91,8 @@ materialization: RunMetadataProviderMessageFragment_StepMaterializationEvent_materialization; } -export interface RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry { - __typename: "EventFloatMetadataEntry"; +export interface RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry" | "EventFloatMetadataEntry"; label: string; description: string | null; } @@ -140,7 +140,7 @@ name: string; } -export type RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries = RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventFloatMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry; +export type RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries = RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventAddressMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult_metadataEntries_EventPythonArtifactMetadataEntry; export interface RunMetadataProviderMessageFragment_StepExpectationResultEvent_expectationResult { __typename: "ExpectationResult"; @@ -165,6 +165,13 @@ path: string; } +export interface RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry"; + label: string; + description: string | null; + address: string | null; +} + export interface RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry { __typename: "EventJsonMetadataEntry"; label: string; @@ -208,7 +215,7 @@ value: number; } -export type RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; +export type RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries = RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPathMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventAddressMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventJsonMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventUrlMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventTextMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventMarkdownMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventPythonArtifactMetadataEntry | RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult_metadataEntries_EventFloatMetadataEntry; export interface RunMetadataProviderMessageFragment_ObjectStoreOperationEvent_operationResult { __typename: "ObjectStoreOperationResult"; diff --git a/js_modules/dagit/src/types/TempMetadataEntryFragment.ts b/js_modules/dagit/src/types/TempMetadataEntryFragment.ts --- a/js_modules/dagit/src/types/TempMetadataEntryFragment.ts +++ b/js_modules/dagit/src/types/TempMetadataEntryFragment.ts @@ -7,8 +7,8 @@ // GraphQL fragment: TempMetadataEntryFragment // ==================================================== -export interface TempMetadataEntryFragment_EventFloatMetadataEntry { - __typename: "EventFloatMetadataEntry"; +export interface TempMetadataEntryFragment_EventAddressMetadataEntry { + __typename: "EventAddressMetadataEntry" | "EventFloatMetadataEntry"; label: string; description: string | null; } @@ -56,4 +56,4 @@ name: string; } -export type TempMetadataEntryFragment = TempMetadataEntryFragment_EventFloatMetadataEntry | TempMetadataEntryFragment_EventPathMetadataEntry | TempMetadataEntryFragment_EventJsonMetadataEntry | TempMetadataEntryFragment_EventUrlMetadataEntry | TempMetadataEntryFragment_EventTextMetadataEntry | TempMetadataEntryFragment_EventMarkdownMetadataEntry | TempMetadataEntryFragment_EventPythonArtifactMetadataEntry; +export type TempMetadataEntryFragment = TempMetadataEntryFragment_EventAddressMetadataEntry | TempMetadataEntryFragment_EventPathMetadataEntry | TempMetadataEntryFragment_EventJsonMetadataEntry | TempMetadataEntryFragment_EventUrlMetadataEntry | TempMetadataEntryFragment_EventTextMetadataEntry | TempMetadataEntryFragment_EventMarkdownMetadataEntry | TempMetadataEntryFragment_EventPythonArtifactMetadataEntry; diff --git a/js_modules/dagit/src/types/globalTypes.ts b/js_modules/dagit/src/types/globalTypes.ts --- a/js_modules/dagit/src/types/globalTypes.ts +++ b/js_modules/dagit/src/types/globalTypes.ts @@ -31,8 +31,10 @@ export enum ObjectStoreOperationType { CP_OBJECT = "CP_OBJECT", + GET_EXTERNAL_OBJECT = "GET_EXTERNAL_OBJECT", GET_OBJECT = "GET_OBJECT", RM_OBJECT = "RM_OBJECT", + SET_EXTERNAL_OBJECT = "SET_EXTERNAL_OBJECT", SET_OBJECT = "SET_OBJECT", } diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py --- a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py @@ -10,6 +10,7 @@ from dagster import PipelineRun, check, seven from dagster.core.definitions.events import ( + AddressMetadataEntryData, EventMetadataEntry, FloatMetadataEntryData, JsonMetadataEntryData, @@ -429,6 +430,14 @@ path = dauphin.NonNull(dauphin.String) +class DauphinEventAddressMetadataEntry(dauphin.ObjectType): + class Meta(object): + name = "EventAddressMetadataEntry" + interfaces = (DauphinEventMetadataEntry,) + + address = dauphin.Field(dauphin.String) + + class DauphinEventJsonMetadataEntry(dauphin.ObjectType): class Meta(object): name = "EventJsonMetadataEntry" @@ -487,6 +496,12 @@ description=metadata_entry.description, path=metadata_entry.entry_data.path, ) + elif isinstance(metadata_entry.entry_data, AddressMetadataEntryData): + yield DauphinEventAddressMetadataEntry( + label=metadata_entry.label, + description=metadata_entry.description, + address=metadata_entry.entry_data.address, + ) elif isinstance(metadata_entry.entry_data, JsonMetadataEntryData): yield DauphinEventJsonMetadataEntry( label=metadata_entry.label, @@ -540,7 +555,9 @@ name = "ObjectStoreOperationType" SET_OBJECT = "SET_OBJECT" + SET_EXTERNAL_OBJECT = "SET_EXTERNAL_OBJECT" GET_OBJECT = "GET_OBJECT" + GET_EXTERNAL_OBJECT = "GET_EXTERNAL_OBJECT" RM_OBJECT = "RM_OBJECT" CP_OBJECT = "CP_OBJECT" diff --git a/python_modules/dagster/dagster/core/definitions/events.py b/python_modules/dagster/dagster/core/definitions/events.py --- a/python_modules/dagster/dagster/core/definitions/events.py +++ b/python_modules/dagster/dagster/core/definitions/events.py @@ -5,6 +5,7 @@ from enum import Enum from dagster import check +from dagster.core.definitions.address import Address from dagster.core.errors import DagsterInvalidAssetKey from dagster.serdes import Persistable, whitelist_for_persistence from dagster.utils.backcompat import experimental_arg_warning @@ -163,6 +164,21 @@ """ return EventMetadataEntry(label, description, PathMetadataEntryData(path)) + @staticmethod + def address(address, label, step_output_handle=None, description=None): + """Static constructor for a metadata entry containing a path as + :py:class:`AddressMetadataEntryData`. + + Args: + address (Optional[str]): The address contained by this metadata entry. + label (str): Short display label for this metadata entry. + step_output_handle + description (Optional[str]): A human-readable description of this metadata entry. + """ + return EventMetadataEntry( + label, description, AddressMetadataEntryData(address, step_output_handle) + ) + @staticmethod def fspath(path, label=None, description=None): """Static constructor for a metadata entry containing a filesystem path as @@ -277,6 +293,27 @@ ) +@whitelist_for_persistence +class AddressMetadataEntryData( + namedtuple("_AddressMetadataEntryData", "address step_output_handle"), Persistable +): + """Container class for address metadata entry data. + + Args: + address (Optional[str]): The path as a string. + step_output_handle + """ + + def __new__(cls, address, step_output_handle=None): + from dagster.core.execution.plan.objects import StepOutputHandle + + return super(AddressMetadataEntryData, cls).__new__( + cls, + check.opt_inst_param(address, "address", Address), + check.opt_inst_param(step_output_handle, "step_output_handle", StepOutputHandle), + ) + + @whitelist_for_persistence class JsonMetadataEntryData(namedtuple("_JsonMetadataEntryData", "data"), Persistable): """Container class for JSON metadata entry data. @@ -333,6 +370,7 @@ TextMetadataEntryData, UrlMetadataEntryData, PathMetadataEntryData, + AddressMetadataEntryData, JsonMetadataEntryData, MarkdownMetadataEntryData, PythonArtifactMetadataEntryData, @@ -368,7 +406,7 @@ cls, value, check.str_param(output_name, "output_name"), - check.opt_str_param(address, "address"), + check.opt_inst_param(address, "address", Address), ) @@ -624,7 +662,9 @@ class ObjectStoreOperationType(Enum): SET_OBJECT = "SET_OBJECT" + SET_EXTERNAL_OBJECT = "SET_EXTERNAL_OBJECT" GET_OBJECT = "GET_OBJECT" + GET_EXTERNAL_OBJECT = "GET_EXTERNAL_OBJECT" RM_OBJECT = "RM_OBJECT" CP_OBJECT = "CP_OBJECT" @@ -632,7 +672,7 @@ class ObjectStoreOperation( namedtuple( "_ObjectStoreOperation", - "op key dest_key obj serialization_strategy_name object_store_name value_name", + "op key dest_key obj serialization_strategy_name object_store_name value_name address step_output_handle", ) ): """This event is used internally by Dagster machinery when values are written to and read from @@ -649,6 +689,8 @@ employed by the operation object_store_name (Optional[str]): The name of the object store that performed the operation. + address + step_output_handle """ def __new__( @@ -660,7 +702,11 @@ serialization_strategy_name=None, object_store_name=None, value_name=None, + address=None, + step_output_handle=None, ): + from dagster.core.execution.plan.objects import StepOutputHandle + return super(ObjectStoreOperation, cls).__new__( cls, op=op, @@ -672,6 +718,10 @@ ), object_store_name=check.opt_str_param(object_store_name, "object_store_name"), value_name=check.opt_str_param(value_name, "value_name"), + address=check.opt_inst_param(address, "address", Address), + step_output_handle=check.opt_inst_param( + step_output_handle, "step_output_handle", StepOutputHandle + ), ) @classmethod @@ -686,6 +736,8 @@ "serialization_strategy_name": inst.serialization_strategy_name, "object_store_name": inst.object_store_name, "value_name": inst.value_name, + "address": inst.address, + "step_output_handle": inst.step_output_handle, }, **kwargs ) diff --git a/python_modules/dagster/dagster/core/events/__init__.py b/python_modules/dagster/dagster/core/events/__init__.py --- a/python_modules/dagster/dagster/core/events/__init__.py +++ b/python_modules/dagster/dagster/core/events/__init__.py @@ -355,6 +355,17 @@ def is_engine_event(self): return self.event_type == DagsterEventType.ENGINE_EVENT + @property + def is_external_operation_event(self): + return ( + self.event_type_value == DagsterEventType.OBJECT_STORE_OPERATION.value + and self.event_specific_data.op + in ( + ObjectStoreOperationType.SET_EXTERNAL_OBJECT.value, + ObjectStoreOperationType.GET_EXTERNAL_OBJECT.value, + ) + ) + @property def asset_key(self): if self.event_type != DagsterEventType.STEP_MATERIALIZATION: @@ -780,7 +791,12 @@ op=object_store_operation_result.op, value_name=value_name, metadata_entries=[ - EventMetadataEntry.path(object_store_operation_result.key, label="key") + EventMetadataEntry.path(object_store_operation_result.key, label="key"), + EventMetadataEntry.address( + address=object_store_operation_result.address, + label="address", + step_output_handle=object_store_operation_result.step_output_handle, + ), ], ), message=message, diff --git a/python_modules/dagster/dagster/core/execution/context_creation_pipeline.py b/python_modules/dagster/dagster/core/execution/context_creation_pipeline.py --- a/python_modules/dagster/dagster/core/execution/context_creation_pipeline.py +++ b/python_modules/dagster/dagster/core/execution/context_creation_pipeline.py @@ -106,7 +106,7 @@ namedtuple( "_ContextCreationData", "pipeline environment_config pipeline_run mode_def system_storage_def " - "intermediate_storage_def executor_def instance resource_keys_to_init", + "intermediate_storage_def executor_def instance resource_keys_to_init external_intermediates", ) ): @property @@ -137,6 +137,7 @@ resource_keys_to_init=get_required_resource_keys_to_init( execution_plan, system_storage_def, intermediate_storage_def ), + external_intermediates=instance.get_external_intermediates(pipeline_run.parent_run_id), ) @@ -395,11 +396,12 @@ ): check.inst_param(context_creation_data, "context_creation_data", ContextCreationData) - environment_config, pipeline_def, system_storage_def, pipeline_run = ( + environment_config, pipeline_def, system_storage_def, pipeline_run, external_intermediates = ( context_creation_data.environment_config, context_creation_data.pipeline_def, context_creation_data.system_storage_def, context_creation_data.pipeline_run, + context_creation_data.external_intermediates, ) system_storage_data = ( @@ -420,6 +422,7 @@ resources=scoped_resources_builder.build( context_creation_data.system_storage_def.required_resource_keys, ), + external_intermediates=external_intermediates, ) ) ) @@ -432,11 +435,18 @@ ): check.inst_param(context_creation_data, "context_creation_data", ContextCreationData) - environment_config, pipeline_def, intermediate_storage_def, pipeline_run = ( + ( + environment_config, + pipeline_def, + intermediate_storage_def, + pipeline_run, + external_intermediates, + ) = ( context_creation_data.environment_config, context_creation_data.pipeline_def, context_creation_data.intermediate_storage_def, context_creation_data.pipeline_run, + context_creation_data.external_intermediates, ) intermediate_storage_data = ( intermediate_storage_data @@ -456,6 +466,7 @@ resources=scoped_resources_builder.build( context_creation_data.intermediate_storage_def.required_resource_keys, ), + external_intermediates=external_intermediates, ) ) ) diff --git a/python_modules/dagster/dagster/core/execution/memoization.py b/python_modules/dagster/dagster/core/execution/memoization.py --- a/python_modules/dagster/dagster/core/execution/memoization.py +++ b/python_modules/dagster/dagster/core/execution/memoization.py @@ -58,16 +58,17 @@ output_handles_from_previous_run ) output_handles_to_copy_by_step = defaultdict(list) + for handle in output_handles_to_copy: output_handles_to_copy_by_step[handle.step_key].append(handle) intermediate_storage = pipeline_context.intermediate_storage + for step in execution_plan.topological_steps(): step_context = pipeline_context.for_step(step) for handle in output_handles_to_copy_by_step.get(step.key, []): if intermediate_storage.has_intermediate(pipeline_context, handle): continue - operation = intermediate_storage.copy_intermediate_from_run( pipeline_context, parent_run_id, handle ) @@ -84,6 +85,7 @@ write_ops = ( ObjectStoreOperationType.SET_OBJECT.value, + ObjectStoreOperationType.SET_EXTERNAL_OBJECT.value, ObjectStoreOperationType.CP_OBJECT.value, ) return ( diff --git a/python_modules/dagster/dagster/core/execution/plan/execute_step.py b/python_modules/dagster/dagster/core/execution/plan/execute_step.py --- a/python_modules/dagster/dagster/core/execution/plan/execute_step.py +++ b/python_modules/dagster/dagster/core/execution/plan/execute_step.py @@ -1,3 +1,5 @@ +import inspect + from dagster import check from dagster.core.definitions import ( AssetMaterialization, @@ -8,6 +10,7 @@ RetryRequested, TypeCheck, ) +from dagster.core.definitions.address import Address from dagster.core.errors import ( DagsterExecutionStepExecutionError, DagsterInvariantViolationError, @@ -321,10 +324,10 @@ step_output_handle = StepOutputHandle.from_step(step=step, output_name=output.output_name) - for evt in _set_intermediates(step_context, step_output, step_output_handle, output): + for evt in _create_output_materializations(step_context, output.output_name, output.value): yield evt - for evt in _create_output_materializations(step_context, output.output_name, output.value): + for evt in _set_intermediates(step_context, step_output, step_output_handle, output): yield evt @@ -334,11 +337,25 @@ dagster_type=step_output.dagster_type, step_output_handle=step_output_handle, value=output.value, + address=output.address, ) + if isinstance(res, ObjectStoreOperation): + # handle event generated by object_store yield DagsterEvent.object_store_operation( step_context, ObjectStoreOperation.serializable(res, value_name=output.output_name) ) + elif inspect.isgenerator(res): + # when external write method (e.g. materializer, config_value) is provided via Output, + # we will handle external materializing events here + for evt in res: + if isinstance(evt, ObjectStoreOperation): + yield DagsterEvent.object_store_operation( + step_context, + ObjectStoreOperation.serializable(evt, value_name=output.output_name), + ) + if isinstance(evt, AssetMaterialization): + yield DagsterEvent.step_materialization(step_context, evt) def _create_output_materializations(step_context, output_name, value): @@ -358,6 +375,8 @@ config_output_name, output_spec = list(output_spec.items())[0] if config_output_name == output_name: step_output = step.step_output_named(output_name) + address = Address(config_value=output_spec) + step_output_handle = StepOutputHandle(step_context.step.key, step_output.name) with user_code_error_boundary( DagsterTypeMaterializationError, msg_fn=lambda: """Error occurred during output materialization: @@ -372,12 +391,22 @@ solid=step_context.solid.name, ), ): - materializations = step_output.dagster_type.materializer.materialize_runtime_values( - step_context, output_spec, value + res = step_context.intermediate_storage.set_intermediate( + context=step_context, + dagster_type=step_output.dagster_type, + step_output_handle=step_output_handle, + value=value, + address=address, ) - for materialization in materializations: - if not isinstance(materialization, (AssetMaterialization, Materialization)): + for event in res: + if isinstance(event, ObjectStoreOperation): + yield DagsterEvent.object_store_operation( + step_context, + ObjectStoreOperation.serializable(event, value_name=step_output.name), + ) + continue + if not isinstance(event, (AssetMaterialization, Materialization)): raise DagsterInvariantViolationError( ( "materialize_runtime_values on type {type_name} has returned " @@ -385,12 +414,12 @@ "AssetMaterialization." ).format( type_name=step_output.dagster_type.name, - value=repr(materialization), - python_type=type(materialization).__name__, + value=repr(event), + python_type=type(event).__name__, ) ) - yield DagsterEvent.step_materialization(step_context, materialization) + yield DagsterEvent.step_materialization(step_context, event) def _user_event_sequence_for_step_compute_fn(step_context, evaluated_inputs): diff --git a/python_modules/dagster/dagster/core/execution/plan/objects.py b/python_modules/dagster/dagster/core/execution/plan/objects.py --- a/python_modules/dagster/dagster/core/execution/plan/objects.py +++ b/python_modules/dagster/dagster/core/execution/plan/objects.py @@ -2,8 +2,14 @@ from enum import Enum from dagster import check -from dagster.core.definitions import AssetMaterialization, Materialization, Solid, SolidHandle -from dagster.core.definitions.events import EventMetadataEntry +from dagster.core.definitions import ( + Address, + AssetMaterialization, + EventMetadataEntry, + Materialization, + Solid, + SolidHandle, +) from dagster.core.types.dagster_type import DagsterType from dagster.serdes import whitelist_for_serdes from dagster.utils import merge_dicts @@ -89,7 +95,7 @@ (AssetMaterialization, Materialization), ), type_check_data=check.opt_inst_param(type_check_data, "type_check_data", TypeCheckData), - address=check.opt_str_param(address, "address"), + address=check.opt_inst_param(address, "address", Address), version=check.opt_str_param(version, "version"), ) diff --git a/python_modules/dagster/dagster/core/instance/__init__.py b/python_modules/dagster/dagster/core/instance/__init__.py --- a/python_modules/dagster/dagster/core/instance/__init__.py +++ b/python_modules/dagster/dagster/core/instance/__init__.py @@ -13,7 +13,7 @@ from dagster import check, seven from dagster.config import Field, Permissive -from dagster.core.definitions.events import AssetKey +from dagster.core.definitions.events import AddressMetadataEntryData, AssetKey from dagster.core.definitions.pipeline import PipelineDefinition, PipelineSubsetDefinition from dagster.core.errors import ( DagsterInvalidConfigError, @@ -491,6 +491,38 @@ def get_run_group(self, run_id): return self._run_storage.get_run_group(run_id) + def get_external_intermediates(self, run_id): + from dagster.core.execution.plan.objects import StepOutputHandle + + if run_id is None: + return + event_logs = self.all_logs(run_id) + + external_intermediates = {} + for record in event_logs: + if not record.is_dagster_event: + continue + + if not record.dagster_event.is_external_operation_event: + continue + + if ( + not record.dagster_event.event_specific_data + or not record.dagster_event.event_specific_data.metadata_entries + ): + continue + + for entry in record.dagster_event.event_specific_data.metadata_entries: + if isinstance(entry.entry_data, AddressMetadataEntryData): + step_output_handle = entry.entry_data.step_output_handle or StepOutputHandle( + record.dagster_event.step_key, + record.dagster_event.event_specific_data.value_name, + ) + external_intermediates[step_output_handle] = entry.entry_data.address + break + + return external_intermediates + def resolve_unmemoized_steps(self, execution_plan, run_config, mode): """ Returns: diff --git a/python_modules/dagster/dagster/core/storage/init.py b/python_modules/dagster/dagster/core/storage/init.py --- a/python_modules/dagster/dagster/core/storage/init.py +++ b/python_modules/dagster/dagster/core/storage/init.py @@ -7,6 +7,7 @@ PipelineDefinition, SystemStorageDefinition, ) +from dagster.core.execution.plan.objects import StepOutputHandle from dagster.core.instance import DagsterInstance from dagster.core.storage.pipeline_run import PipelineRun from dagster.core.storage.type_storage import TypeStoragePluginRegistry @@ -18,7 +19,7 @@ "InitSystemStorageContext", ( "pipeline_def mode_def system_storage_def pipeline_run instance environment_config " - "type_storage_plugin_registry resources system_storage_config" + "type_storage_plugin_registry resources system_storage_config external_intermediates" ), ) ): @@ -38,6 +39,9 @@ system_storage_config (Dict[str, Any]): The system storage-specific configuration data provided by the environment config. The schema for this data is defined by the ``config_field`` argument to :py:class:`SystemStorageDefinition`. + external_intermediates (Dict[StepOutputHandle, Address]): The mapping from step output to + address which tracks the intermediates that have been stored outside the system + intermediate directory. """ def __new__( @@ -51,6 +55,7 @@ type_storage_plugin_registry, resources, system_storage_config, + external_intermediates, ): return super(InitSystemStorageContext, cls).__new__( cls, @@ -73,6 +78,9 @@ system_storage_config=check.dict_param( system_storage_config, system_storage_config, key_type=str ), + external_intermediates=check.opt_dict_param( + external_intermediates, "external_intermediates", key_type=StepOutputHandle + ), ) @@ -81,7 +89,7 @@ "InitIntermediateStorageContext", ( "pipeline_def mode_def intermediate_storage_def pipeline_run instance environment_config " - "type_storage_plugin_registry resources intermediate_storage_config" + "type_storage_plugin_registry resources intermediate_storage_config external_intermediates" ), ) ): @@ -101,6 +109,9 @@ intermediate_storage_config (Dict[str, Any]): The intermediate storage-specific configuration data provided by the environment config. The schema for this data is defined by the ``config_field`` argument to :py:class:`IntermediateStorageDefinition`. + external_intermediates (Dict[StepOutputHandle, Address]): The mapping from step output to + address which tracks the intermediates that have been stored outside the system + intermediate directory. """ def __new__( @@ -114,6 +125,7 @@ type_storage_plugin_registry, resources, intermediate_storage_config, + external_intermediates, ): return super(InitIntermediateStorageContext, cls).__new__( cls, @@ -136,4 +148,7 @@ intermediate_storage_config=check.dict_param( intermediate_storage_config, intermediate_storage_config, key_type=str ), + external_intermediates=check.opt_dict_param( + external_intermediates, "external_intermediates", key_type=StepOutputHandle + ), ) diff --git a/python_modules/dagster/dagster/core/storage/intermediate_storage.py b/python_modules/dagster/dagster/core/storage/intermediate_storage.py --- a/python_modules/dagster/dagster/core/storage/intermediate_storage.py +++ b/python_modules/dagster/dagster/core/storage/intermediate_storage.py @@ -4,7 +4,13 @@ from dagster import check from dagster.core.definitions.address import Address -from dagster.core.errors import DagsterAddressIOError +from dagster.core.definitions.events import ObjectStoreOperation, ObjectStoreOperationType +from dagster.core.errors import ( + DagsterAddressIOError, + DagsterTypeLoadingError, + DagsterTypeMaterializationError, + user_code_error_boundary, +) from dagster.core.execution.context.system import SystemExecutionContext from dagster.core.execution.plan.objects import StepOutputHandle from dagster.core.types.dagster_type import DagsterType, resolve_dagster_type @@ -66,13 +72,26 @@ class ObjectStoreIntermediateStorage(IntermediateStorage): - def __init__(self, object_store, root_for_run_id, run_id, type_storage_plugin_registry): + def __init__( + self, + object_store, + root_for_run_id, + run_id, + type_storage_plugin_registry, + external_intermediates=None, + ): self.root_for_run_id = check.callable_param(root_for_run_id, "root_for_run_id") self.run_id = check.str_param(run_id, "run_id") self.object_store = check.inst_param(object_store, "object_store", ObjectStore) self.type_storage_plugin_registry = check.inst_param( type_storage_plugin_registry, "type_storage_plugin_registry", TypeStoragePluginRegistry ) + self.external_intermediates = check.opt_dict_param( + external_intermediates, + "external_intermediates", + key_type=StepOutputHandle, + value_type=Address, + ) def _get_paths(self, step_output_handle): return ["intermediates", step_output_handle.step_key, step_output_handle.output_name] @@ -109,6 +128,39 @@ # if address is provided, the intermediate could be stored outside so we skip the check check.invariant(self.has_intermediate(context, step_output_handle)) + # if the intermediate was stored externally + if address is None: + address = self.external_intermediates.get(step_output_handle) + + if address and address.config_value and dagster_type.loader: + with user_code_error_boundary( + DagsterTypeLoadingError, + msg_fn=lambda: ( + "Error occurred during input loading:" + "input name: '{output_name}'" + "step key: '{key}'" + "solid invocation: '{solid}'" + "solid definition: '{solid_def}'" + ).format( + output_name=step_output_handle.output_name, # FIXME input name instead + key=context.step.key, + solid_def=context.solid_def.name, + solid=context.solid.name, + ), + ): + value = dagster_type.loader.construct_from_config_value( + context, address.config_value + ) + # yield "get external object" operation event for cross-run intermediate storage + return ObjectStoreOperation( + op=ObjectStoreOperationType.GET_EXTERNAL_OBJECT, + key=address.key, + obj=value, + address=address, + step_output_handle=step_output_handle, + ) + + # START: to deprecate https://github.com/dagster-io/dagster/issues/3043 if self.type_storage_plugin_registry.is_registered(dagster_type): return self.type_storage_plugin_registry.get(dagster_type.name).get_intermediate_object( self, context, dagster_type, step_output_handle @@ -117,6 +169,7 @@ self.type_storage_plugin_registry.check_for_unsupported_composite_overrides( dagster_type ) + # END: to deprecate return self.get_intermediate_object(dagster_type, step_output_handle, address) @@ -141,6 +194,37 @@ except (IOError, OSError) as e: raise DagsterAddressIOError(str(e)) + def _set_external_intermediate(self, context, dagster_type, step_output_handle, value, address): + with user_code_error_boundary( + DagsterTypeMaterializationError, + msg_fn=lambda: ( + "Error occurred during output materialization:" + "output name: '{output_name}'" + "step key: '{key}'" + "solid invocation: '{solid}'" + "solid definition: '{solid_def}'" + ).format( + output_name=step_output_handle.output_name, + key=context.step.key, + solid_def=context.solid_def.name, + solid=context.solid.name, + ), + ): + materializations = dagster_type.materializer.materialize_runtime_values( + context, address.config_value, value + ) + self.external_intermediates[step_output_handle] = address + for materialization in materializations: + yield materialization + + # yield "set external object" operation event for cross-run intermediate storage + yield ObjectStoreOperation( + op=ObjectStoreOperationType.SET_EXTERNAL_OBJECT, + key=address.key, + address=address, + step_output_handle=step_output_handle, + ) + def set_intermediate( self, context, dagster_type=None, step_output_handle=None, value=None, address=None, ): @@ -158,6 +242,23 @@ % (step_output_handle.step_key, step_output_handle.output_name) ) + # load to some external address + if address and address.config_value and dagster_type.materializer: + return self._set_external_intermediate( + context, dagster_type, step_output_handle, value, address + ) + + # skip if the intermediate has already been set by type materializer + if step_output_handle in self.external_intermediates: + context.log.info( + ( + "{step_output_handle} has already been materialized by DagsterTypeMaterializer. " + "Skip object store." + ).format(step_output_handle=step_output_handle) + ) + return + + # START: to deprecate https://github.com/dagster-io/dagster/issues/3043 if self.type_storage_plugin_registry.is_registered(dagster_type): return self.type_storage_plugin_registry.get(dagster_type.name).set_intermediate_object( self, context, dagster_type, step_output_handle, value @@ -166,6 +267,7 @@ self.type_storage_plugin_registry.check_for_unsupported_composite_overrides( dagster_type ) + # END: to deprecate return self.set_intermediate_object(dagster_type, step_output_handle, value, address) @@ -177,7 +279,9 @@ check.param_invariant(len(paths) > 0, "paths") key = self.object_store.key_for_paths([self.root] + paths) - return self.object_store.has_object(key) + return ( + self.object_store.has_object(key) or step_output_handle in self.external_intermediates + ) def rm_intermediate(self, context, step_output_handle): check.opt_inst_param(context, "context", SystemExecutionContext) @@ -229,7 +333,9 @@ ) -def build_fs_intermediate_storage(root_for_run_id, run_id, type_storage_plugin_registry=None): +def build_fs_intermediate_storage( + root_for_run_id, run_id, type_storage_plugin_registry=None, external_intermediates=None +): return ObjectStoreIntermediateStorage( FilesystemObjectStore(), root_for_run_id, @@ -237,4 +343,5 @@ type_storage_plugin_registry if type_storage_plugin_registry else TypeStoragePluginRegistry(types_to_register=[]), + external_intermediates, ) diff --git a/python_modules/dagster/dagster/core/storage/system_storage.py b/python_modules/dagster/dagster/core/storage/system_storage.py --- a/python_modules/dagster/dagster/core/storage/system_storage.py +++ b/python_modules/dagster/dagster/core/storage/system_storage.py @@ -50,6 +50,7 @@ type_storage_plugin_registry=init_context.type_storage_plugin_registry if init_context.type_storage_plugin_registry else TypeStoragePluginRegistry(types_to_register=[]), + external_intermediates=init_context.external_intermediates, ) @@ -115,6 +116,7 @@ root_for_run_id=lambda _: override_dir, run_id=init_context.pipeline_run.run_id, type_storage_plugin_registry=init_context.type_storage_plugin_registry, + external_intermediates=init_context.external_intermediates, ) else: file_manager = LocalFileManager.for_instance( @@ -124,6 +126,7 @@ init_context.instance.intermediates_directory, run_id=init_context.pipeline_run.run_id, type_storage_plugin_registry=init_context.type_storage_plugin_registry, + external_intermediates=init_context.external_intermediates, ) return SystemStorageData(file_manager=file_manager, intermediate_storage=intermediate_storage,) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_required_resources.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_required_resources.py --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_required_resources.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_required_resources.py @@ -469,6 +469,13 @@ resources_initted["a"] = True yield "A" + @dagster_type_loader( + String, required_resource_keys={"a"} if should_require_resources else set() + ) + def load(context, hello): + assert context.resources.a == "A" + return hello + @dagster_type_materializer( String, required_resource_keys={"a"} if should_require_resources else set() ) @@ -476,7 +483,7 @@ assert context.resources.a == "A" return AssetMaterialization("hello") - CustomDagsterType = create_any_type(name="CustomType", materializer=materialize) + CustomDagsterType = create_any_type(name="CustomType", materializer=materialize, loader=load) @solid(output_defs=[OutputDefinition(CustomDagsterType)]) def output_solid(_context): diff --git a/python_modules/dagster/dagster_tests/core_tests/test_addresses.py b/python_modules/dagster/dagster_tests/core_tests/test_addresses.py --- a/python_modules/dagster/dagster_tests/core_tests/test_addresses.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_addresses.py @@ -1,13 +1,31 @@ import os +import pickle import pytest -from dagster import Int, seven +from dagster import ( + AssetMaterialization, + InputDefinition, + Int, + Output, + OutputDefinition, + String, + dagster_type_loader, + dagster_type_materializer, + execute_pipeline, + reexecute_pipeline, + seven, + solid, +) +from dagster.core.definitions import pipeline from dagster.core.definitions.address import Address +from dagster.core.definitions.events import AddressMetadataEntryData from dagster.core.errors import DagsterAddressIOError from dagster.core.execution.plan.objects import StepOutputHandle from dagster.core.instance import DagsterInstance from dagster.core.storage.intermediate_storage import build_fs_intermediate_storage +from dagster.core.storage.object_store import ObjectStoreOperationType +from dagster.core.types.dagster_type import create_any_type def test_address_path_operation_using_intermediates_file_system(): @@ -61,3 +79,150 @@ step_output_handle=StepOutputHandle("solid1.compute"), address=Address(path=os.path.join(tmpdir_path, "invalid.output")), ) + + +@dagster_type_loader(String) +def my_loader(_, path): + with open(path, "rb") as fd: + return pickle.load(fd) + + +@dagster_type_materializer(String) +def my_materializer(_, path, value): + with open(path, "wb") as f: + pickle.dump(value, f) + return AssetMaterialization.file(path) + + +CustomType = create_any_type(name="MyType", materializer=my_materializer, loader=my_loader) + + +def test_address_config_value_operation_run_config(): + @solid( + input_defs=[InputDefinition("val", CustomType)], output_defs=[OutputDefinition(CustomType)] + ) + def solid_a(_, val): + return val + + @solid(input_defs=[InputDefinition("val", CustomType)]) + def solid_b(_, val): + assert val == [1, 2, 3] + + @pipeline + def foo(): + solid_b(solid_a()) + + instance = DagsterInstance.ephemeral() + with seven.TemporaryDirectory() as tempdir: + data = [1, 2, 3] + file_path = os.path.join(tempdir, "foo") + + with open(file_path, "wb") as f: + pickle.dump(data, f) + intermediate_output_path = os.path.join(tempdir, "intermediate") + run_config = { + "solids": { + "solid_a": { + "inputs": {"val": file_path}, + "outputs": [{"result": intermediate_output_path}], + } + }, + "storage": {"filesystem": None}, + } + result = execute_pipeline(foo, run_config=run_config, instance=instance,) + + assert result.success + intermediate_val = result.result_for_solid("solid_a").output_value() + assert intermediate_val == [1, 2, 3] + + # make sure data is passed by reference between solids + external_intermediate_events = list( + filter(lambda evt: evt.is_external_operation_event, result.event_list) + ) + assert len(external_intermediate_events) == 2 + # SET_EXTERNAL_OBJECT + assert ( + external_intermediate_events[0].event_specific_data.op + == ObjectStoreOperationType.SET_EXTERNAL_OBJECT.value + ) + for entry_data in external_intermediate_events[1].event_specific_data.metadata_entries: + if isinstance(entry_data, AddressMetadataEntryData): + assert entry_data.address.config_value == intermediate_output_path + # GET_EXTERNAL_OBJECT + assert ( + external_intermediate_events[1].event_specific_data.op + == ObjectStoreOperationType.GET_EXTERNAL_OBJECT.value + ) + for entry_data in external_intermediate_events[1].event_specific_data.metadata_entries: + if isinstance(entry_data, AddressMetadataEntryData): + assert entry_data.address.config_value == intermediate_output_path + + # test cross-run external_intermediates + reexecution_result = reexecute_pipeline( + foo, + result.run_id, + instance=instance, + run_config=run_config, + step_selection=["solid_b.compute"], + ) + + assert reexecution_result.success + + get_external_object_events = list( + filter(lambda evt: evt.is_external_operation_event, reexecution_result.event_list) + ) + assert len(get_external_object_events) == 1 + assert ( + get_external_object_events[0].event_specific_data.op + == ObjectStoreOperationType.GET_EXTERNAL_OBJECT.value + ) + + +def test_address_config_value_operation_in_solid(): + with seven.TemporaryDirectory() as tempdir: + intermediate_output_path = os.path.join(tempdir, "intermediate") + + @solid(output_defs=[OutputDefinition(CustomType)]) + def solid_a(_): + return Output([1, 2, 3], address=Address(config_value=intermediate_output_path)) + + @solid(input_defs=[InputDefinition("val", CustomType)]) + def solid_b(_, val): + assert val == [1, 2, 3] + + @pipeline + def foo(): + solid_b(solid_a()) + + result = execute_pipeline(foo) + assert result.success + + # verify output is also the intermediate + with open(os.path.join(tempdir, "intermediate"), "rb") as fd: + output_val = pickle.load(fd) + + intermediate_val = result.result_for_solid("solid_a").output_value() + assert intermediate_val == [1, 2, 3] + assert intermediate_val == output_val + + # make sure data is passed by reference between solids + external_intermediate_events = list( + filter(lambda evt: evt.is_external_operation_event, result.event_list) + ) + assert len(external_intermediate_events) == 2 + # SET_EXTERNAL_OBJECT + assert ( + external_intermediate_events[0].event_specific_data.op + == ObjectStoreOperationType.SET_EXTERNAL_OBJECT.value + ) + for entry_data in external_intermediate_events[1].event_specific_data.metadata_entries: + if isinstance(entry_data, AddressMetadataEntryData): + assert entry_data.address.config_value == intermediate_output_path + # GET_EXTERNAL_OBJECT + assert ( + external_intermediate_events[1].event_specific_data.op + == ObjectStoreOperationType.GET_EXTERNAL_OBJECT.value + ) + for entry_data in external_intermediate_events[1].event_specific_data.metadata_entries: + if isinstance(entry_data, AddressMetadataEntryData): + assert entry_data.address.config_value == intermediate_output_path diff --git a/python_modules/libraries/dagster-pandas/dagster_pandas_tests/test_data_frame.py b/python_modules/libraries/dagster-pandas/dagster_pandas_tests/test_data_frame.py --- a/python_modules/libraries/dagster-pandas/dagster_pandas_tests/test_data_frame.py +++ b/python_modules/libraries/dagster-pandas/dagster_pandas_tests/test_data_frame.py @@ -1,3 +1,5 @@ +import os + import pytest from dagster_pandas.constraints import ( ColumnDTypeInSetConstraint, @@ -13,17 +15,16 @@ DagsterInvariantViolationError, DagsterType, EventMetadataEntry, - Field, InputDefinition, Output, OutputDefinition, - Selector, check_dagster_type, dagster_type_loader, dagster_type_materializer, execute_pipeline, execute_solid, pipeline, + seven, solid, ) from dagster.utils import safe_tempfile_path @@ -197,24 +198,14 @@ def test_custom_dagster_dataframe_parametrizable_input(): - @dagster_type_loader( - Selector({"door_a": Field(str), "door_b": Field(str), "door_c": Field(str),}) - ) - def silly_loader(_, config): - which_door = list(config.keys())[0] - if which_door == "door_a": - return DataFrame({"foo": ["goat"]}) - elif which_door == "door_b": - return DataFrame({"foo": ["car"]}) - elif which_door == "door_c": - return DataFrame({"foo": ["goat"]}) - raise DagsterInvariantViolationError( - "You did not pick a door. You chose: {which_door}".format(which_door=which_door) - ) + @dagster_type_loader(str) + def silly_loader(_, path): + return read_csv(path) - @dagster_type_materializer(Selector({"devnull": Field(str), "nothing": Field(str)})) - def silly_materializer(_, _config, _value): - return AssetMaterialization(asset_key="nothing", description="just one of those days") + @dagster_type_materializer(str) + def silly_materializer(_, path, df): + df.to_csv(path) + return AssetMaterialization.file(path) TestDataFrame = create_dagster_pandas_dataframe_type( name="TestDataFrame", @@ -230,21 +221,33 @@ def did_i_win(_, df): return df - solid_result = execute_solid( - did_i_win, - run_config={ - "solids": { - "did_i_win": { - "inputs": {"df": {"door_a": "bar"}}, - "outputs": [{"result": {"devnull": "baz"}}], + with seven.TemporaryDirectory() as tempdir: + file_path = os.path.join(tempdir, "foo.csv") + output_path = os.path.join(tempdir, "output.csv") + foo_df = DataFrame({"foo": [1, 2, 3]}) + foo_df.to_csv(file_path) + + solid_result = execute_solid( + did_i_win, + run_config={ + "solids": { + "did_i_win": { + "inputs": {"df": file_path}, + "outputs": [{"result": output_path}], + } } - } - }, - ) - assert solid_result.success - output_df = solid_result.output_value() - assert isinstance(output_df, DataFrame) - assert output_df["foo"].tolist() == ["goat"] - materialization_events = solid_result.materialization_events_during_compute - assert len(materialization_events) == 1 - assert materialization_events[0].event_specific_data.materialization.label == "nothing" + }, + ) + + assert solid_result.success + output_df = solid_result.output_value() + assert isinstance(output_df, DataFrame) + assert output_df["foo"].tolist() == foo_df["foo"].tolist() + materialization_events = solid_result.materialization_events_during_compute + assert len(materialization_events) == 1 + assert ( + materialization_events[0] + .event_specific_data.materialization.metadata_entries[0] + .entry_data.path + == output_path + )