From cabfffd855cef2f69dfece98f92d10c177628564 Mon Sep 17 00:00:00 2001 From: David Liu Date: Mon, 18 Nov 2024 16:21:37 -0500 Subject: [PATCH 01/10] relabel to test sensor/schedule to evaluate tick --- .../packages/ui-core/src/schedules/ScheduleDetails.tsx | 2 +- .../dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx b/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx index ad4cdf9592c43..0c93dc35c4701 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/schedules/ScheduleDetails.tsx @@ -67,7 +67,7 @@ export const ScheduleDetails = (props: { setShowTestTickDialog(true); }} > - Test Schedule + Evaluate tick } diff --git a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx index b51301ac76417..e1dfccab4bf9c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx @@ -125,7 +125,7 @@ export const SensorDetails = ({ setShowTestTickDialog(true); }} > - Test sensor + Evaluate tick From 990c2be458af20670f5d0ce98ca3e600a953bd57 Mon Sep 17 00:00:00 2001 From: David Liu Date: Wed, 16 Oct 2024 17:40:17 -0400 Subject: [PATCH 02/10] add launch all button --- .../packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx | 7 ++++++- .../packages/ui-core/src/ticks/SensorDryRunDialog.tsx | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx index 5509208e5d6f4..b8249d6a81275 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx @@ -182,7 +182,12 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { ); } else { - return ; + return ( + <> + + + + ); } }, [onClose, shouldEvaluate]); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index b955dde5a967c..0af78ed4cea37 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -124,6 +124,7 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop if (sensorExecutionData || error) { return ( + + } /> - { - setShowTestTickDialog(false); - }} - name={schedule.name} - repoAddress={repoAddress} - jobName={pipelineName} - /> {schedule.description ? ( diff --git a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx index e1dfccab4bf9c..2e2d005dac948 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/sensors/SensorDetails.tsx @@ -17,6 +17,7 @@ import {EditCursorDialog} from './EditCursorDialog'; import {SensorMonitoredAssets} from './SensorMonitoredAssets'; import {SensorResetButton} from './SensorResetButton'; import {SensorSwitch} from './SensorSwitch'; +import {EvaluateTickButtonSensor} from '../ticks/EvaluateTickButtonSensor'; import {SensorFragment} from './types/SensorFragment.types'; import {usePermissionsForLocation} from '../app/Permissions'; import {QueryRefreshCountdown, QueryRefreshState} from '../app/QueryRefresh'; @@ -25,7 +26,6 @@ import {AutomationAssetSelectionFragment} from '../automation/types/AutomationAs import {InstigationStatus, SensorType} from '../graphql/types'; import {RepositoryLink} from '../nav/RepositoryLink'; import {TimestampDisplay} from '../schedules/TimestampDisplay'; -import {SensorDryRunDialog} from '../ticks/SensorDryRunDialog'; import {TickStatusTag} from '../ticks/TickStatusTag'; import {RepoAddress} from '../workspace/types'; @@ -92,7 +92,6 @@ export const SensorDetails = ({ sensor.sensorState.typeSpecificData.__typename === 'SensorData' && sensor.sensorState.typeSpecificData.lastCursor; - const [showTestTickDialog, setShowTestTickDialog] = useState(false); const running = status === InstigationStatus.RUNNING; return ( @@ -114,33 +113,16 @@ export const SensorDetails = ({ right={ - - - + } /> - { - setShowTestTickDialog(false); - }} - currentCursor={cursor || ''} - name={sensor.name} - repoAddress={repoAddress} - jobName={sensor.targets?.[0]?.pipelineName || ''} - /> {sensor.description ? ( diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx new file mode 100644 index 0000000000000..03ea49e3fdf2d --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx @@ -0,0 +1,41 @@ +import {Box, Button} from '@dagster-io/ui-components'; +import {useState} from 'react'; + +import {EvaluateScheduleDialog} from './EvaluateScheduleDialog'; +import {RepoAddress} from '../workspace/types'; + +interface EvaluateTickButtonScheduleProps { + name: string; + repoAddress: RepoAddress; + jobName: string; +} + +export const EvaluateTickButtonSchedule = ({ + name, + repoAddress, + jobName, +}: EvaluateTickButtonScheduleProps) => { + const [showTestTickDialog, setShowTestTickDialog] = useState(false); + + return ( + + + { + setShowTestTickDialog(false); + }} + name={name} + repoAddress={repoAddress} + jobName={jobName} + /> + + ); +}; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx new file mode 100644 index 0000000000000..cec8337183c97 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx @@ -0,0 +1,49 @@ +import {Box, Button, Tooltip} from '@dagster-io/ui-components'; +import {useState} from 'react'; + +import {SensorDryRunDialog} from './SensorDryRunDialog'; +import {SensorType} from '../graphql/types'; +import {RepoAddress} from '../workspace/types'; + +interface EvaluateTickButtonSensorProps { + cursor: string; + name: string; + repoAddress: RepoAddress; + jobName: string; + sensorType: SensorType; +} + +export const EvaluateTickButtonSensor = ({ + cursor, + name, + repoAddress, + jobName, + sensorType, +}: EvaluateTickButtonSensorProps) => { + const [showTestTickDialog, setShowTestTickDialog] = useState(false); + + return ( + + + + + setShowTestTickDialog(false)} + currentCursor={cursor} + name={name} + repoAddress={repoAddress} + jobName={jobName} + /> + + ); +}; From 54db7e504be72c4a7c4c57facd6d02400ad5b68f Mon Sep 17 00:00:00 2001 From: David Liu Date: Thu, 5 Dec 2024 16:38:42 -0500 Subject: [PATCH 04/10] add launch all backend functionality --- .../src/graphql/possibleTypes.generated.json | 2 +- .../ui-core/src/graphql/schema.graphql | 11 + .../packages/ui-core/src/graphql/types.ts | 55 ++++ .../dagster_graphql/client/query.py | 65 +++++ .../dagster_graphql/schema/roots/__init__.py | 2 + .../dagster_graphql/schema/roots/mutation.py | 37 ++- .../dagster_graphql/schema/runs.py | 21 +- .../graphql/test_run_launcher.py | 243 +++++++++++++++++- 8 files changed, 425 insertions(+), 11 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json b/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json index 24d3627e786cf..25046b8064e0f 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/possibleTypes.generated.json @@ -1 +1 @@ -{"DisplayableEvent":["EngineEvent","ExecutionStepOutputEvent","ExpectationResult","FailureMetadata","HandledOutputEvent","LoadedInputEvent","ObjectStoreOperationResult","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","TypeCheck"],"MarkerEvent":["EngineEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent"],"ErrorEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepUpForRetryEvent","HookErroredEvent","RunCanceledEvent","RunFailureEvent","ResourceInitFailureEvent"],"MessageEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","LogMessageEvent","ObjectStoreOperationEvent","RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetMaterializationPlannedEvent","LogsCapturedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"RunEvent":["RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","AssetMaterializationPlannedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent"],"PipelineRunStepStats":["RunStepStats"],"StepEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","ObjectStoreOperationEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"AssetOwner":["UserAssetOwner","TeamAssetOwner"],"AssetPartitionStatuses":["DefaultPartitionStatuses","MultiPartitionStatuses","TimePartitionStatuses"],"PartitionStatus1D":["TimePartitionStatuses","DefaultPartitionStatuses"],"AssetChecksOrError":["AssetChecks","AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError"],"Instigator":["Schedule","Sensor"],"EvaluationStackEntry":["EvaluationStackListItemEntry","EvaluationStackPathEntry","EvaluationStackMapKeyEntry","EvaluationStackMapValueEntry"],"IPipelineSnapshot":["Pipeline","PipelineSnapshot","Job"],"PipelineConfigValidationError":["FieldNotDefinedConfigError","FieldsNotDefinedConfigError","MissingFieldConfigError","MissingFieldsConfigError","RuntimeMismatchConfigError","SelectorTypeConfigError"],"PipelineConfigValidationInvalid":["RunConfigValidationInvalid"],"PipelineConfigValidationResult":["InvalidSubsetError","PipelineConfigValidationValid","RunConfigValidationInvalid","PipelineNotFoundError","PythonError"],"PipelineReference":["PipelineSnapshot","UnknownPipeline"],"PipelineRun":["Run"],"DagsterRunEvent":["ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","ExecutionStepRestartEvent","LogMessageEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunFailureEvent","RunStartEvent","RunEnqueuedEvent","RunDequeuedEvent","RunStartingEvent","RunCancelingEvent","RunCanceledEvent","RunSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","HandledOutputEvent","LoadedInputEvent","LogsCapturedEvent","ObjectStoreOperationEvent","StepExpectationResultEvent","MaterializationEvent","ObservationEvent","EngineEvent","HookCompletedEvent","HookSkippedEvent","HookErroredEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetMaterializationPlannedEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"PipelineRunLogsSubscriptionPayload":["PipelineRunLogsSubscriptionSuccess","PipelineRunLogsSubscriptionFailure"],"RunOrError":["Run","RunNotFoundError","PythonError"],"PipelineRunStatsSnapshot":["RunStatsSnapshot"],"RunStatsSnapshotOrError":["RunStatsSnapshot","PythonError"],"PipelineSnapshotOrError":["PipelineNotFoundError","PipelineSnapshot","PipelineSnapshotNotFoundError","PythonError"],"RunsFeedEntry":["Run","PartitionBackfill"],"AssetOrError":["Asset","AssetNotFoundError"],"AssetsOrError":["AssetConnection","PythonError"],"DeletePipelineRunResult":["DeletePipelineRunSuccess","UnauthorizedError","PythonError","RunNotFoundError"],"ExecutionPlanOrError":["ExecutionPlan","RunConfigValidationInvalid","PipelineNotFoundError","InvalidSubsetError","PythonError"],"PipelineOrError":["Pipeline","PipelineNotFoundError","InvalidSubsetError","PythonError"],"ReloadRepositoryLocationMutationResult":["WorkspaceLocationEntry","ReloadNotSupported","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"RepositoryLocationOrLoadError":["RepositoryLocation","PythonError"],"ReloadWorkspaceMutationResult":["Workspace","UnauthorizedError","PythonError"],"ShutdownRepositoryLocationMutationResult":["ShutdownRepositoryLocationSuccess","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"TerminatePipelineExecutionFailure":["TerminateRunFailure"],"TerminatePipelineExecutionSuccess":["TerminateRunSuccess"],"TerminateRunResult":["TerminateRunSuccess","TerminateRunFailure","RunNotFoundError","UnauthorizedError","PythonError"],"ScheduleMutationResult":["PythonError","UnauthorizedError","ScheduleStateResult","ScheduleNotFoundError"],"ScheduleOrError":["Schedule","ScheduleNotFoundError","PythonError"],"SchedulerOrError":["Scheduler","SchedulerNotDefinedError","PythonError"],"SchedulesOrError":["Schedules","RepositoryNotFoundError","PythonError"],"ScheduleTickSpecificData":["ScheduleTickSuccessData","ScheduleTickFailureData"],"LaunchBackfillResult":["LaunchBackfillSuccess","PartitionSetNotFoundError","PartitionKeysNotFoundError","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"ConfigTypeOrError":["EnumConfigType","CompositeConfigType","RegularConfigType","PipelineNotFoundError","ConfigTypeNotFoundError","PythonError"],"ConfigType":["ArrayConfigType","CompositeConfigType","EnumConfigType","NullableConfigType","RegularConfigType","ScalarUnionConfigType","MapConfigType"],"WrappingConfigType":["ArrayConfigType","NullableConfigType"],"DagsterType":["ListDagsterType","NullableDagsterType","RegularDagsterType"],"DagsterTypeOrError":["RegularDagsterType","PipelineNotFoundError","DagsterTypeNotFoundError","PythonError"],"WrappingDagsterType":["ListDagsterType","NullableDagsterType"],"Error":["AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError","PartitionKeysNotFoundError","AssetNotFoundError","ConflictingExecutionParamsError","ConfigTypeNotFoundError","DagsterTypeNotFoundError","InvalidPipelineRunsFilterError","InvalidSubsetError","ModeNotFoundError","NoModeProvidedError","PartitionSetNotFoundError","PipelineNotFoundError","RunConflict","PipelineSnapshotNotFoundError","PresetNotFoundError","PythonError","ErrorChainLink","UnauthorizedError","ReloadNotSupported","RepositoryLocationNotFound","RepositoryNotFoundError","ResourceNotFoundError","RunGroupNotFoundError","RunNotFoundError","ScheduleNotFoundError","SchedulerNotDefinedError","SensorNotFoundError","UnsupportedOperationError","DuplicateDynamicPartitionError","InstigationStateNotFoundError","SolidStepStatusUnavailableError","GraphNotFoundError","BackfillNotFoundError","PartitionSubsetDeserializationError","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PipelineRunConflict":["RunConflict"],"PipelineRunNotFoundError":["RunNotFoundError"],"RepositoriesOrError":["RepositoryConnection","RepositoryNotFoundError","PythonError"],"RepositoryOrError":["PythonError","Repository","RepositoryNotFoundError"],"WorkspaceLocationEntryOrError":["WorkspaceLocationEntry","PythonError"],"InstigationTypeSpecificData":["SensorData","ScheduleData"],"InstigationStateOrError":["InstigationState","InstigationStateNotFoundError","PythonError"],"InstigationStatesOrError":["InstigationStates","PythonError"],"MetadataEntry":["TableColumnLineageMetadataEntry","TableSchemaMetadataEntry","TableMetadataEntry","FloatMetadataEntry","IntMetadataEntry","JsonMetadataEntry","BoolMetadataEntry","MarkdownMetadataEntry","PathMetadataEntry","NotebookMetadataEntry","PythonArtifactMetadataEntry","TextMetadataEntry","UrlMetadataEntry","PipelineRunMetadataEntry","AssetMetadataEntry","JobMetadataEntry","CodeReferencesMetadataEntry","NullMetadataEntry","TimestampMetadataEntry"],"SourceLocation":["LocalFileCodeReference","UrlCodeReference"],"PartitionRunConfigOrError":["PartitionRunConfig","PythonError"],"AssetBackfillStatus":["AssetPartitionsStatusCounts","UnpartitionedAssetStatus"],"PartitionSetOrError":["PartitionSet","PartitionSetNotFoundError","PythonError"],"PartitionSetsOrError":["PartitionSets","PipelineNotFoundError","PythonError"],"PartitionsOrError":["Partitions","PythonError"],"PartitionStatusesOrError":["PartitionStatuses","PythonError"],"PartitionTagsOrError":["PartitionTags","PythonError"],"RunConfigSchemaOrError":["RunConfigSchema","PipelineNotFoundError","InvalidSubsetError","ModeNotFoundError","PythonError"],"LaunchRunResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchRunReexecutionResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchPipelineRunSuccess":["LaunchRunSuccess"],"RunsOrError":["Runs","InvalidPipelineRunsFilterError","PythonError"],"PipelineRuns":["Runs"],"RunGroupOrError":["RunGroup","RunGroupNotFoundError","PythonError"],"SensorOrError":["Sensor","SensorNotFoundError","UnauthorizedError","PythonError"],"SensorsOrError":["Sensors","RepositoryNotFoundError","PythonError"],"StopSensorMutationResultOrError":["StopSensorMutationResult","UnauthorizedError","PythonError"],"ISolidDefinition":["CompositeSolidDefinition","SolidDefinition"],"SolidContainer":["Pipeline","PipelineSnapshot","Job","CompositeSolidDefinition","Graph"],"SolidStepStatsOrError":["SolidStepStatsConnection","SolidStepStatusUnavailableError"],"WorkspaceOrError":["Workspace","PythonError"],"WorkspaceLocationStatusEntriesOrError":["WorkspaceLocationStatusEntries","PythonError"],"ResourcesOrError":["ResourceConnection","PipelineNotFoundError","InvalidSubsetError","PythonError"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourceDetailsListOrError":["ResourceDetailsList","RepositoryNotFoundError","PythonError"],"EnvVarWithConsumersOrError":["EnvVarWithConsumersList","PythonError"],"RunsFeedConnectionOrError":["RunsFeedConnection","PythonError"],"RunsFeedCountOrError":["RunsFeedCount","PythonError"],"RunTagKeysOrError":["PythonError","RunTagKeys"],"RunTagsOrError":["PythonError","RunTags"],"RunIdsOrError":["RunIds","InvalidPipelineRunsFilterError","PythonError"],"AssetNodeOrError":["AssetNode","AssetNotFoundError"],"PartitionBackfillOrError":["PartitionBackfill","BackfillNotFoundError","PythonError"],"PartitionBackfillsOrError":["PartitionBackfills","PythonError"],"EventConnectionOrError":["EventConnection","RunNotFoundError","PythonError"],"AutoMaterializeAssetEvaluationRecordsOrError":["AutoMaterializeAssetEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PartitionKeysOrError":["PartitionKeys","PartitionSubsetDeserializationError"],"AutoMaterializeRuleEvaluationData":["TextRuleEvaluationData","ParentMaterializedRuleEvaluationData","WaitingOnKeysRuleEvaluationData"],"AssetConditionEvaluationNode":["UnpartitionedAssetConditionEvaluationNode","PartitionedAssetConditionEvaluationNode","SpecificPartitionAssetConditionEvaluationNode"],"AssetConditionEvaluationRecordsOrError":["AssetConditionEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"SensorDryRunResult":["PythonError","SensorNotFoundError","DryRunInstigationTick"],"ScheduleDryRunResult":["DryRunInstigationTick","PythonError","ScheduleNotFoundError"],"TerminateRunsResultOrError":["TerminateRunsResult","PythonError"],"AssetWipeMutationResult":["AssetNotFoundError","UnauthorizedError","PythonError","UnsupportedOperationError","AssetWipeSuccess"],"ReportRunlessAssetEventsResult":["UnauthorizedError","PythonError","ReportRunlessAssetEventsSuccess"],"ResumeBackfillResult":["ResumeBackfillSuccess","UnauthorizedError","PythonError"],"CancelBackfillResult":["CancelBackfillSuccess","UnauthorizedError","PythonError"],"LogTelemetryMutationResult":["LogTelemetrySuccess","PythonError"],"AddDynamicPartitionResult":["AddDynamicPartitionSuccess","UnauthorizedError","PythonError","DuplicateDynamicPartitionError"],"DeleteDynamicPartitionsResult":["DeleteDynamicPartitionsSuccess","UnauthorizedError","PythonError"]} \ No newline at end of file +{"DisplayableEvent":["EngineEvent","ExecutionStepOutputEvent","ExpectationResult","FailureMetadata","HandledOutputEvent","LoadedInputEvent","ObjectStoreOperationResult","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","TypeCheck"],"MarkerEvent":["EngineEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent"],"ErrorEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepUpForRetryEvent","HookErroredEvent","RunCanceledEvent","RunFailureEvent","ResourceInitFailureEvent"],"MessageEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","LogMessageEvent","ObjectStoreOperationEvent","RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetMaterializationPlannedEvent","LogsCapturedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"RunEvent":["RunCanceledEvent","RunCancelingEvent","RunDequeuedEvent","RunEnqueuedEvent","RunFailureEvent","RunStartEvent","RunStartingEvent","RunSuccessEvent","AssetMaterializationPlannedEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent"],"PipelineRunStepStats":["RunStepStats"],"StepEvent":["EngineEvent","ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepRestartEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","HandledOutputEvent","HookCompletedEvent","HookErroredEvent","HookSkippedEvent","LoadedInputEvent","ObjectStoreOperationEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","StepExpectationResultEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","MaterializationEvent","ObservationEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"AssetOwner":["UserAssetOwner","TeamAssetOwner"],"AssetPartitionStatuses":["DefaultPartitionStatuses","MultiPartitionStatuses","TimePartitionStatuses"],"PartitionStatus1D":["TimePartitionStatuses","DefaultPartitionStatuses"],"AssetChecksOrError":["AssetChecks","AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError"],"Instigator":["Schedule","Sensor"],"EvaluationStackEntry":["EvaluationStackListItemEntry","EvaluationStackPathEntry","EvaluationStackMapKeyEntry","EvaluationStackMapValueEntry"],"IPipelineSnapshot":["Pipeline","PipelineSnapshot","Job"],"PipelineConfigValidationError":["FieldNotDefinedConfigError","FieldsNotDefinedConfigError","MissingFieldConfigError","MissingFieldsConfigError","RuntimeMismatchConfigError","SelectorTypeConfigError"],"PipelineConfigValidationInvalid":["RunConfigValidationInvalid"],"PipelineConfigValidationResult":["InvalidSubsetError","PipelineConfigValidationValid","RunConfigValidationInvalid","PipelineNotFoundError","PythonError"],"PipelineReference":["PipelineSnapshot","UnknownPipeline"],"PipelineRun":["Run"],"DagsterRunEvent":["ExecutionStepFailureEvent","ExecutionStepInputEvent","ExecutionStepOutputEvent","ExecutionStepSkippedEvent","ExecutionStepStartEvent","ExecutionStepSuccessEvent","ExecutionStepUpForRetryEvent","ExecutionStepRestartEvent","LogMessageEvent","ResourceInitFailureEvent","ResourceInitStartedEvent","ResourceInitSuccessEvent","RunFailureEvent","RunStartEvent","RunEnqueuedEvent","RunDequeuedEvent","RunStartingEvent","RunCancelingEvent","RunCanceledEvent","RunSuccessEvent","StepWorkerStartedEvent","StepWorkerStartingEvent","HandledOutputEvent","LoadedInputEvent","LogsCapturedEvent","ObjectStoreOperationEvent","StepExpectationResultEvent","MaterializationEvent","ObservationEvent","EngineEvent","HookCompletedEvent","HookSkippedEvent","HookErroredEvent","AlertStartEvent","AlertSuccessEvent","AlertFailureEvent","AssetMaterializationPlannedEvent","AssetCheckEvaluationPlannedEvent","AssetCheckEvaluationEvent"],"PipelineRunLogsSubscriptionPayload":["PipelineRunLogsSubscriptionSuccess","PipelineRunLogsSubscriptionFailure"],"RunOrError":["Run","RunNotFoundError","PythonError"],"PipelineRunStatsSnapshot":["RunStatsSnapshot"],"RunStatsSnapshotOrError":["RunStatsSnapshot","PythonError"],"PipelineSnapshotOrError":["PipelineNotFoundError","PipelineSnapshot","PipelineSnapshotNotFoundError","PythonError"],"RunsFeedEntry":["Run","PartitionBackfill"],"AssetOrError":["Asset","AssetNotFoundError"],"AssetsOrError":["AssetConnection","PythonError"],"DeletePipelineRunResult":["DeletePipelineRunSuccess","UnauthorizedError","PythonError","RunNotFoundError"],"ExecutionPlanOrError":["ExecutionPlan","RunConfigValidationInvalid","PipelineNotFoundError","InvalidSubsetError","PythonError"],"LaunchMultipleRunsResultOrError":["LaunchMultipleRunsResult","PythonError"],"PipelineOrError":["Pipeline","PipelineNotFoundError","InvalidSubsetError","PythonError"],"ReloadRepositoryLocationMutationResult":["WorkspaceLocationEntry","ReloadNotSupported","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"RepositoryLocationOrLoadError":["RepositoryLocation","PythonError"],"ReloadWorkspaceMutationResult":["Workspace","UnauthorizedError","PythonError"],"ShutdownRepositoryLocationMutationResult":["ShutdownRepositoryLocationSuccess","RepositoryLocationNotFound","UnauthorizedError","PythonError"],"TerminatePipelineExecutionFailure":["TerminateRunFailure"],"TerminatePipelineExecutionSuccess":["TerminateRunSuccess"],"TerminateRunResult":["TerminateRunSuccess","TerminateRunFailure","RunNotFoundError","UnauthorizedError","PythonError"],"ScheduleMutationResult":["PythonError","UnauthorizedError","ScheduleStateResult","ScheduleNotFoundError"],"ScheduleOrError":["Schedule","ScheduleNotFoundError","PythonError"],"SchedulerOrError":["Scheduler","SchedulerNotDefinedError","PythonError"],"SchedulesOrError":["Schedules","RepositoryNotFoundError","PythonError"],"ScheduleTickSpecificData":["ScheduleTickSuccessData","ScheduleTickFailureData"],"LaunchBackfillResult":["LaunchBackfillSuccess","PartitionSetNotFoundError","PartitionKeysNotFoundError","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"ConfigTypeOrError":["EnumConfigType","CompositeConfigType","RegularConfigType","PipelineNotFoundError","ConfigTypeNotFoundError","PythonError"],"ConfigType":["ArrayConfigType","CompositeConfigType","EnumConfigType","NullableConfigType","RegularConfigType","ScalarUnionConfigType","MapConfigType"],"WrappingConfigType":["ArrayConfigType","NullableConfigType"],"DagsterType":["ListDagsterType","NullableDagsterType","RegularDagsterType"],"DagsterTypeOrError":["RegularDagsterType","PipelineNotFoundError","DagsterTypeNotFoundError","PythonError"],"WrappingDagsterType":["ListDagsterType","NullableDagsterType"],"Error":["AssetCheckNeedsMigrationError","AssetCheckNeedsUserCodeUpgrade","AssetCheckNeedsAgentUpgradeError","PartitionKeysNotFoundError","AssetNotFoundError","ConflictingExecutionParamsError","ConfigTypeNotFoundError","DagsterTypeNotFoundError","InvalidPipelineRunsFilterError","InvalidSubsetError","ModeNotFoundError","NoModeProvidedError","PartitionSetNotFoundError","PipelineNotFoundError","RunConflict","PipelineSnapshotNotFoundError","PresetNotFoundError","PythonError","ErrorChainLink","UnauthorizedError","ReloadNotSupported","RepositoryLocationNotFound","RepositoryNotFoundError","ResourceNotFoundError","RunGroupNotFoundError","RunNotFoundError","ScheduleNotFoundError","SchedulerNotDefinedError","SensorNotFoundError","UnsupportedOperationError","DuplicateDynamicPartitionError","InstigationStateNotFoundError","SolidStepStatusUnavailableError","GraphNotFoundError","BackfillNotFoundError","PartitionSubsetDeserializationError","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PipelineRunConflict":["RunConflict"],"PipelineRunNotFoundError":["RunNotFoundError"],"RepositoriesOrError":["RepositoryConnection","RepositoryNotFoundError","PythonError"],"RepositoryOrError":["PythonError","Repository","RepositoryNotFoundError"],"WorkspaceLocationEntryOrError":["WorkspaceLocationEntry","PythonError"],"InstigationTypeSpecificData":["SensorData","ScheduleData"],"InstigationStateOrError":["InstigationState","InstigationStateNotFoundError","PythonError"],"InstigationStatesOrError":["InstigationStates","PythonError"],"MetadataEntry":["TableColumnLineageMetadataEntry","TableSchemaMetadataEntry","TableMetadataEntry","FloatMetadataEntry","IntMetadataEntry","JsonMetadataEntry","BoolMetadataEntry","MarkdownMetadataEntry","PathMetadataEntry","NotebookMetadataEntry","PythonArtifactMetadataEntry","TextMetadataEntry","UrlMetadataEntry","PipelineRunMetadataEntry","AssetMetadataEntry","JobMetadataEntry","CodeReferencesMetadataEntry","NullMetadataEntry","TimestampMetadataEntry"],"SourceLocation":["LocalFileCodeReference","UrlCodeReference"],"PartitionRunConfigOrError":["PartitionRunConfig","PythonError"],"AssetBackfillStatus":["AssetPartitionsStatusCounts","UnpartitionedAssetStatus"],"PartitionSetOrError":["PartitionSet","PartitionSetNotFoundError","PythonError"],"PartitionSetsOrError":["PartitionSets","PipelineNotFoundError","PythonError"],"PartitionsOrError":["Partitions","PythonError"],"PartitionStatusesOrError":["PartitionStatuses","PythonError"],"PartitionTagsOrError":["PartitionTags","PythonError"],"RunConfigSchemaOrError":["RunConfigSchema","PipelineNotFoundError","InvalidSubsetError","ModeNotFoundError","PythonError"],"LaunchRunResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchRunReexecutionResult":["LaunchRunSuccess","InvalidStepError","InvalidOutputError","RunConfigValidationInvalid","PipelineNotFoundError","RunConflict","UnauthorizedError","PythonError","InvalidSubsetError","PresetNotFoundError","ConflictingExecutionParamsError","NoModeProvidedError"],"LaunchPipelineRunSuccess":["LaunchRunSuccess"],"RunsOrError":["Runs","InvalidPipelineRunsFilterError","PythonError"],"PipelineRuns":["Runs"],"RunGroupOrError":["RunGroup","RunGroupNotFoundError","PythonError"],"SensorOrError":["Sensor","SensorNotFoundError","UnauthorizedError","PythonError"],"SensorsOrError":["Sensors","RepositoryNotFoundError","PythonError"],"StopSensorMutationResultOrError":["StopSensorMutationResult","UnauthorizedError","PythonError"],"ISolidDefinition":["CompositeSolidDefinition","SolidDefinition"],"SolidContainer":["Pipeline","PipelineSnapshot","Job","CompositeSolidDefinition","Graph"],"SolidStepStatsOrError":["SolidStepStatsConnection","SolidStepStatusUnavailableError"],"WorkspaceOrError":["Workspace","PythonError"],"WorkspaceLocationStatusEntriesOrError":["WorkspaceLocationStatusEntries","PythonError"],"ResourcesOrError":["ResourceConnection","PipelineNotFoundError","InvalidSubsetError","PythonError"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourceDetailsListOrError":["ResourceDetailsList","RepositoryNotFoundError","PythonError"],"EnvVarWithConsumersOrError":["EnvVarWithConsumersList","PythonError"],"RunsFeedConnectionOrError":["RunsFeedConnection","PythonError"],"RunsFeedCountOrError":["RunsFeedCount","PythonError"],"RunTagKeysOrError":["PythonError","RunTagKeys"],"RunTagsOrError":["PythonError","RunTags"],"RunIdsOrError":["RunIds","InvalidPipelineRunsFilterError","PythonError"],"AssetNodeOrError":["AssetNode","AssetNotFoundError"],"PartitionBackfillOrError":["PartitionBackfill","BackfillNotFoundError","PythonError"],"PartitionBackfillsOrError":["PartitionBackfills","PythonError"],"EventConnectionOrError":["EventConnection","RunNotFoundError","PythonError"],"AutoMaterializeAssetEvaluationRecordsOrError":["AutoMaterializeAssetEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"PartitionKeysOrError":["PartitionKeys","PartitionSubsetDeserializationError"],"AutoMaterializeRuleEvaluationData":["TextRuleEvaluationData","ParentMaterializedRuleEvaluationData","WaitingOnKeysRuleEvaluationData"],"AssetConditionEvaluationNode":["UnpartitionedAssetConditionEvaluationNode","PartitionedAssetConditionEvaluationNode","SpecificPartitionAssetConditionEvaluationNode"],"AssetConditionEvaluationRecordsOrError":["AssetConditionEvaluationRecords","AutoMaterializeAssetEvaluationNeedsMigrationError"],"SensorDryRunResult":["PythonError","SensorNotFoundError","DryRunInstigationTick"],"ScheduleDryRunResult":["DryRunInstigationTick","PythonError","ScheduleNotFoundError"],"TerminateRunsResultOrError":["TerminateRunsResult","PythonError"],"AssetWipeMutationResult":["AssetNotFoundError","UnauthorizedError","PythonError","UnsupportedOperationError","AssetWipeSuccess"],"ReportRunlessAssetEventsResult":["UnauthorizedError","PythonError","ReportRunlessAssetEventsSuccess"],"ResumeBackfillResult":["ResumeBackfillSuccess","UnauthorizedError","PythonError"],"CancelBackfillResult":["CancelBackfillSuccess","UnauthorizedError","PythonError"],"LogTelemetryMutationResult":["LogTelemetrySuccess","PythonError"],"AddDynamicPartitionResult":["AddDynamicPartitionSuccess","UnauthorizedError","PythonError","DuplicateDynamicPartitionError"],"DeleteDynamicPartitionsResult":["DeleteDynamicPartitionsSuccess","UnauthorizedError","PythonError"]} \ No newline at end of file diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql index 7fee0c080d122..2f4c3193df0c9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql @@ -1511,6 +1511,12 @@ type LaunchRunMutation { Output: LaunchRunResult! } +type LaunchMultipleRunsMutation { + Output: LaunchMultipleRunsResultOrError! +} + +union LaunchMultipleRunsResultOrError = LaunchMultipleRunsResult | PythonError + type LaunchRunReexecutionMutation { Output: LaunchRunReexecutionResult! } @@ -2980,6 +2986,10 @@ union LaunchRunResult = | ConflictingExecutionParamsError | NoModeProvidedError +type LaunchMultipleRunsResult { + launchMultipleRunsResult: [LaunchRunResult!]! +} + union LaunchRunReexecutionResult = | LaunchRunSuccess | InvalidStepError @@ -3704,6 +3714,7 @@ type AutomationConditionEvaluationNode { type Mutation { launchPipelineExecution(executionParams: ExecutionParams!): LaunchRunResult! launchRun(executionParams: ExecutionParams!): LaunchRunResult! + launchMultipleRuns(executionParamsList: [ExecutionParams!]!): LaunchMultipleRunsResultOrError! launchPipelineReexecution( executionParams: ExecutionParams reexecutionParams: ReexecutionParams diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts index dffa563c8dcdd..ed57441d95d09 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts @@ -2206,6 +2206,18 @@ export type LaunchBackfillSuccess = { launchedRunIds: Maybe>>; }; +export type LaunchMultipleRunsMutation = { + __typename: 'LaunchMultipleRunsMutation'; + Output: LaunchMultipleRunsResultOrError; +}; + +export type LaunchMultipleRunsResult = { + __typename: 'LaunchMultipleRunsResult'; + launchMultipleRunsResult: Array; +}; + +export type LaunchMultipleRunsResultOrError = LaunchMultipleRunsResult | PythonError; + export type LaunchPipelineRunSuccess = { run: Run; }; @@ -2614,6 +2626,7 @@ export type Mutation = { deleteRun: DeletePipelineRunResult; freeConcurrencySlots: Scalars['Boolean']['output']; freeConcurrencySlotsForRun: Scalars['Boolean']['output']; + launchMultipleRuns: LaunchMultipleRunsResultOrError; launchPartitionBackfill: LaunchBackfillResult; launchPipelineExecution: LaunchRunResult; launchPipelineReexecution: LaunchRunReexecutionResult; @@ -2681,6 +2694,10 @@ export type MutationFreeConcurrencySlotsForRunArgs = { runId: Scalars['String']['input']; }; +export type MutationLaunchMultipleRunsArgs = { + executionParamsList: Array; +}; + export type MutationLaunchPartitionBackfillArgs = { backfillParams: LaunchBackfillParams; }; @@ -9441,6 +9458,38 @@ export const buildLaunchBackfillSuccess = ( }; }; +export const buildLaunchMultipleRunsMutation = ( + overrides?: Partial, + _relationshipsToOmit: Set = new Set(), +): {__typename: 'LaunchMultipleRunsMutation'} & LaunchMultipleRunsMutation => { + const relationshipsToOmit: Set = new Set(_relationshipsToOmit); + relationshipsToOmit.add('LaunchMultipleRunsMutation'); + return { + __typename: 'LaunchMultipleRunsMutation', + Output: + overrides && overrides.hasOwnProperty('Output') + ? overrides.Output! + : relationshipsToOmit.has('LaunchMultipleRunsResult') + ? ({} as LaunchMultipleRunsResult) + : buildLaunchMultipleRunsResult({}, relationshipsToOmit), + }; +}; + +export const buildLaunchMultipleRunsResult = ( + overrides?: Partial, + _relationshipsToOmit: Set = new Set(), +): {__typename: 'LaunchMultipleRunsResult'} & LaunchMultipleRunsResult => { + const relationshipsToOmit: Set = new Set(_relationshipsToOmit); + relationshipsToOmit.add('LaunchMultipleRunsResult'); + return { + __typename: 'LaunchMultipleRunsResult', + launchMultipleRunsResult: + overrides && overrides.hasOwnProperty('launchMultipleRunsResult') + ? overrides.launchMultipleRunsResult! + : [], + }; +}; + export const buildLaunchPipelineRunSuccess = ( overrides?: Partial, _relationshipsToOmit: Set = new Set(), @@ -10222,6 +10271,12 @@ export const buildMutation = ( overrides && overrides.hasOwnProperty('freeConcurrencySlotsForRun') ? overrides.freeConcurrencySlotsForRun! : false, + launchMultipleRuns: + overrides && overrides.hasOwnProperty('launchMultipleRuns') + ? overrides.launchMultipleRuns! + : relationshipsToOmit.has('LaunchMultipleRunsResult') + ? ({} as LaunchMultipleRunsResult) + : buildLaunchMultipleRunsResult({}, relationshipsToOmit), launchPartitionBackfill: overrides && overrides.hasOwnProperty('launchPartitionBackfill') ? overrides.launchPartitionBackfill! diff --git a/python_modules/dagster-graphql/dagster_graphql/client/query.py b/python_modules/dagster-graphql/dagster_graphql/client/query.py index 579b4e6e7a046..76f5ea6d9607f 100644 --- a/python_modules/dagster-graphql/dagster_graphql/client/query.py +++ b/python_modules/dagster-graphql/dagster_graphql/client/query.py @@ -330,6 +330,71 @@ ) +LAUNCH_MULTIPLE_RUNS_MUTATION = ( + ERROR_FRAGMENT + + """ +mutation($executionParamsList: [ExecutionParams!]!) { + launchMultipleRuns(executionParamsList: $executionParamsList) { + __typename + ... on LaunchMultipleRunsResult { + launchMultipleRunsResult { + __typename + ... on InvalidStepError { + invalidStepKey + } + ... on InvalidOutputError { + stepKey + invalidOutputName + } + ... on LaunchRunSuccess { + run { + runId + pipeline { + name + } + tags { + key + value + } + status + runConfigYaml + mode + resolvedOpSelection + } + } + ... on ConflictingExecutionParamsError { + message + } + ... on PresetNotFoundError { + preset + message + } + ... on RunConfigValidationInvalid { + pipelineName + errors { + __typename + message + path + reason + } + } + ... on PipelineNotFoundError { + message + pipelineName + } + ... on PythonError { + ...errorFragment + } + } + } + ... on PythonError { + ...errorFragment + } + } +} +""" +) + LAUNCH_PIPELINE_REEXECUTION_MUTATION = ( ERROR_FRAGMENT + """ diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py b/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py index 6533d2d0b0f32..10f029145bc11 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/roots/__init__.py @@ -10,6 +10,7 @@ def types(): GrapheneDeletePipelineRunSuccess, GrapheneDeleteRunMutation, GrapheneLaunchBackfillMutation, + GrapheneLaunchMultipleRunsMutation, GrapheneLaunchRunMutation, GrapheneLaunchRunReexecutionMutation, GrapheneReloadRepositoryLocationMutation, @@ -38,6 +39,7 @@ def types(): GrapheneExecutionPlanOrError, GrapheneLaunchBackfillMutation, GrapheneLaunchRunMutation, + GrapheneLaunchMultipleRunsMutation, GrapheneLaunchRunReexecutionMutation, GraphenePipelineOrError, GrapheneReloadRepositoryLocationMutation, diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py b/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py index 0e98ceda547ee..687d1fd70ba60 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/roots/mutation.py @@ -1,4 +1,4 @@ -from typing import Optional, Sequence, Union +from typing import List, Optional, Sequence, Union import dagster._check as check import graphene @@ -78,6 +78,8 @@ ) from dagster_graphql.schema.pipelines.pipeline import GrapheneRun from dagster_graphql.schema.runs import ( + GrapheneLaunchMultipleRunsResult, + GrapheneLaunchMultipleRunsResultOrError, GrapheneLaunchRunReexecutionResult, GrapheneLaunchRunResult, GrapheneLaunchRunSuccess, @@ -316,6 +318,38 @@ def mutate( return create_execution_params_and_launch_pipeline_exec(graphene_info, executionParams) +class GrapheneLaunchMultipleRunsMutation(graphene.Mutation): + """Launches multiple job runs.""" + + Output = graphene.NonNull(GrapheneLaunchMultipleRunsResultOrError) + + class Arguments: + executionParamsList = non_null_list(GrapheneExecutionParams) + + class Meta: + name = "LaunchMultipleRunsMutation" + + @capture_error + def mutate( + self, graphene_info: ResolveInfo, executionParamsList: List[GrapheneExecutionParams] + ) -> Union[ + GrapheneLaunchMultipleRunsResult, + GrapheneError, + GraphenePythonError, + ]: + launch_multiple_runs_result = [] + + for execution_params in executionParamsList: + result = GrapheneLaunchRunMutation.mutate( + None, graphene_info, executionParams=execution_params + ) + launch_multiple_runs_result.append(result) + + return GrapheneLaunchMultipleRunsResult( + launchMultipleRunsResult=launch_multiple_runs_result + ) + + class GrapheneLaunchBackfillMutation(graphene.Mutation): """Launches a set of partition backfill runs.""" @@ -984,6 +1018,7 @@ class Meta: launchPipelineExecution = GrapheneLaunchRunMutation.Field() launchRun = GrapheneLaunchRunMutation.Field() + launchMultipleRuns = GrapheneLaunchMultipleRunsMutation.Field() launchPipelineReexecution = GrapheneLaunchRunReexecutionMutation.Field() launchRunReexecution = GrapheneLaunchRunReexecutionMutation.Field() startSchedule = GrapheneStartScheduleMutation.Field() diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py index 9aabe08512373..971f13d621efa 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/runs.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/runs.py @@ -11,6 +11,7 @@ from dagster_graphql.implementation.fetch_runs import get_run_ids, get_runs, get_runs_count from dagster_graphql.implementation.utils import UserFacingGraphQLError +from dagster_graphql.schema.backfill import pipeline_execution_error_types from dagster_graphql.schema.errors import ( GrapheneInvalidPipelineRunsFilterError, GraphenePythonError, @@ -73,17 +74,28 @@ class Meta: class GrapheneLaunchRunResult(graphene.Union): class Meta: - from dagster_graphql.schema.backfill import pipeline_execution_error_types - types = launch_pipeline_run_result_types + pipeline_execution_error_types name = "LaunchRunResult" -class GrapheneLaunchRunReexecutionResult(graphene.Union): +class GrapheneLaunchMultipleRunsResult(graphene.ObjectType): + """Contains results from multiple pipeline launches.""" + + launchMultipleRunsResult = non_null_list(GrapheneLaunchRunResult) + + class Meta: + name = "LaunchMultipleRunsResult" + + +class GrapheneLaunchMultipleRunsResultOrError(graphene.Union): class Meta: - from dagster_graphql.schema.backfill import pipeline_execution_error_types + types = (GrapheneLaunchMultipleRunsResult, GraphenePythonError) + name = "LaunchMultipleRunsResultOrError" + +class GrapheneLaunchRunReexecutionResult(graphene.Union): + class Meta: types = launch_pipeline_run_result_types + pipeline_execution_error_types name = "LaunchRunReexecutionResult" @@ -213,6 +225,7 @@ def parse_run_config_input( types = [ GrapheneLaunchRunResult, + GrapheneLaunchMultipleRunsResult, GrapheneLaunchRunReexecutionResult, GrapheneLaunchPipelineRunSuccess, GrapheneLaunchRunSuccess, diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py index f33fc6139d7e7..3a58623c301eb 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_run_launcher.py @@ -1,9 +1,13 @@ from typing import Any +from unittest.mock import patch from dagster._core.test_utils import wait_for_runs_to_finish from dagster._core.workspace.context import WorkspaceRequestContext -from dagster_graphql.client.query import LAUNCH_PIPELINE_EXECUTION_MUTATION -from dagster_graphql.test.utils import execute_dagster_graphql, infer_job_selector +from dagster_graphql.client.query import ( + LAUNCH_MULTIPLE_RUNS_MUTATION, + LAUNCH_PIPELINE_EXECUTION_MUTATION, +) +from dagster_graphql.test.utils import GqlResult, execute_dagster_graphql, infer_job_selector from dagster_graphql_tests.graphql.graphql_context_test_suite import ( GraphQLContextVariant, @@ -32,6 +36,12 @@ BaseTestSuite: Any = make_graphql_context_test_suite( context_variants=GraphQLContextVariant.all_executing_variants() ) +LaunchFailTestSuite: Any = make_graphql_context_test_suite( + context_variants=GraphQLContextVariant.all_non_launchable_variants() +) +ReadOnlyTestSuite: Any = make_graphql_context_test_suite( + context_variants=GraphQLContextVariant.all_readonly_variants() +) class TestBasicLaunch(BaseTestSuite): @@ -83,10 +93,99 @@ def test_run_launcher_subset(self, graphql_context: WorkspaceRequestContext): assert result.data["pipelineRunOrError"]["status"] == "SUCCESS" assert result.data["pipelineRunOrError"]["stats"]["stepsSucceeded"] == 1 + def test_run_launcher_unauthorized(self, graphql_context: WorkspaceRequestContext): + selector = infer_job_selector(graphql_context, "no_config_job") -LaunchFailTestSuite: Any = make_graphql_context_test_suite( - context_variants=GraphQLContextVariant.all_non_launchable_variants() -) + with patch.object(graphql_context, "has_permission_for_location", return_value=False): + with patch.object(graphql_context, "was_permission_checked", return_value=True): + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_PIPELINE_EXECUTION_MUTATION, + variables={"executionParams": {"selector": selector, "mode": "default"}}, + ) + assert result.data["launchPipelineExecution"]["__typename"] == "UnauthorizedError" + + +class TestMultipleLaunch(BaseTestSuite): + def test_multiple_run_launcher_same_job(self, graphql_context: WorkspaceRequestContext): + selector = infer_job_selector(graphql_context, "no_config_job") + + # test with multiple of the same job + executionParamsList = [ + {"selector": selector, "mode": "default"}, + {"selector": selector, "mode": "default"}, + {"selector": selector, "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + launches = result.data["launchMultipleRuns"] + + assert launches["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in launches + results = launches["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "LaunchRunSuccess" + + def test_multiple_run_launcher_multiple_jobs(self, graphql_context: WorkspaceRequestContext): + selectors = [ + infer_job_selector(graphql_context, "no_config_job"), + infer_job_selector(graphql_context, "more_complicated_config", ["noop_op"]), + ] + + # test with multiple of the same job + executionParamsList = [ + {"selector": selectors[0], "mode": "default"}, + {"selector": selectors[1], "mode": "default"}, + {"selector": selectors[0], "mode": "default"}, + {"selector": selectors[1], "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + launches = result.data["launchMultipleRuns"] + + assert launches["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in launches + results = launches["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "LaunchRunSuccess" + + def test_multiple_launch_failure_unauthorized(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + # mock no permissions + with patch.object(graphql_context, "has_permission_for_location", return_value=False): + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in result_data + + results = result_data["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "UnauthorizedError" class TestFailedLaunch(LaunchFailTestSuite): @@ -97,6 +196,7 @@ def test_launch_failure(self, graphql_context: WorkspaceRequestContext): query=LAUNCH_PIPELINE_EXECUTION_MUTATION, variables={"executionParams": {"selector": selector, "mode": "default"}}, ) + assert result.data["launchPipelineExecution"]["__typename"] != "LaunchRunSuccess" # fetch the most recent run, which should be this one that just failed to launch @@ -105,7 +205,140 @@ def test_launch_failure(self, graphql_context: WorkspaceRequestContext): result = execute_dagster_graphql( context=graphql_context, query=RUN_QUERY, variables={"runId": run.run_id} ) + assert result.data["pipelineRunOrError"]["__typename"] == "Run" assert result.data["pipelineRunOrError"]["status"] == "FAILURE" assert result.data["pipelineRunOrError"]["startTime"] assert result.data["pipelineRunOrError"]["endTime"] + + +class TestFailedMultipleLaunch(LaunchFailTestSuite): + def test_multiple_launch_failure(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + results = result_data["launchMultipleRunsResult"] + + assert len(results) == 2 + + for run_result in results: + assert run_result["__typename"] == "PythonError" + assert run_result["message"].startswith( + "NotImplementedError: The entire purpose of this is to throw on launch" + ) + assert run_result["className"] == "NotImplementedError" + + +class TestFailedMultipleLaunchReadOnly(ReadOnlyTestSuite): + def test_multiple_launch_failure_readonly(self, graphql_context: WorkspaceRequestContext): + executionParamsList = [ + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + {"selector": infer_job_selector(graphql_context, "no_config_job"), "mode": "default"}, + ] + + result = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + assert "launchMultipleRunsResult" in result_data + + results = result_data["launchMultipleRunsResult"] + + for result in results: + assert result["__typename"] == "UnauthorizedError" + + +class TestSuccessAndFailureMultipleLaunch(BaseTestSuite): + def test_launch_multiple_runs_success_and_failure( + self, graphql_context: WorkspaceRequestContext + ): + launchSuccessExecutionParams = [ + { + "selector": { + "repositoryLocationName": "test", + "repositoryName": "test_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + { + "selector": { + "repositoryLocationName": "test", + "repositoryName": "test_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + ] + + pipelineNotFoundExecutionParams = [ + { + "selector": { + "repositoryLocationName": "test", + "repositoryName": "test_dict_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + { + "selector": { + "repositoryLocationName": "test", + "repositoryName": "test_dict_repo", + "pipelineName": "no_config_job", + "solidSelection": None, + "assetSelection": None, + "assetCheckSelection": None, + }, + "mode": "default", + }, + ] + + executionParamsList = [executionParams for executionParams in launchSuccessExecutionParams] + executionParamsList.extend( + [executionParams for executionParams in pipelineNotFoundExecutionParams] + ) + + result: GqlResult = execute_dagster_graphql( + context=graphql_context, + query=LAUNCH_MULTIPLE_RUNS_MUTATION, + variables={"executionParamsList": executionParamsList}, + ) + + assert "launchMultipleRuns" in result.data + result_data = result.data["launchMultipleRuns"] + + assert result_data["__typename"] == "LaunchMultipleRunsResult" + results = result_data["launchMultipleRunsResult"] + + assert len(results) == 4 + + assert results[0]["__typename"] == "LaunchRunSuccess" + assert results[1]["__typename"] == "LaunchRunSuccess" + assert results[2]["__typename"] == "PipelineNotFoundError" + assert results[3]["__typename"] == "PipelineNotFoundError" From 1c2d10ec6d3cbdde0a71b0bc71ac049781dfe8e1 Mon Sep 17 00:00:00 2001 From: David Liu Date: Thu, 5 Dec 2024 16:42:18 -0500 Subject: [PATCH 05/10] add launch all frontend functionality for sensors --- .../dagster-ui/packages/ui-core/client.json | 1 + .../packages/ui-core/src/app/Telemetry.tsx | 8 +- .../useLaunchMultipleRunsWithTelemetry.ts | 62 +++++++ .../packages/ui-core/src/runs/RunUtils.tsx | 152 +++++++++++++++++- .../ui-core/src/runs/types/RunUtils.types.ts | 102 ++++++++++++ .../ui-core/src/ticks/SensorDryRunDialog.tsx | 80 ++++++++- .../src/util/buildExecutionParamsList.ts | 53 ++++++ 7 files changed, 447 insertions(+), 11 deletions(-) create mode 100644 js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts create mode 100644 js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts diff --git a/js_modules/dagster-ui/packages/ui-core/client.json b/js_modules/dagster-ui/packages/ui-core/client.json index 4c62310e025b4..9b8c430dd2ead 100644 --- a/js_modules/dagster-ui/packages/ui-core/client.json +++ b/js_modules/dagster-ui/packages/ui-core/client.json @@ -126,6 +126,7 @@ "RunRootQuery": "1aa4561b33c2cfb079d7a3ff284096fc3208a46dee748a24c7af827a2cb22919", "RunStatsQuery": "75e80f740a79607de9e1152f9b7074d319197fbc219784c767c1abd5553e9a49", "LaunchPipelineExecution": "292088c4a697aca6be1d3bbc0cfc45d8a13cdb2e75cfedc64b68c6245ea34f89", + "LaunchMultipleRuns": "a56d9efdb35e71e0fd1744dd768129248943bc5b23e717458b82c46829661763", "Delete": "3c61c79b99122910e754a8863e80dc5ed479a0c23cc1a9d9878d91e603fc0dfe", "Terminate": "67acf403eb320a93c9a9aa07f675a1557e0887d499cd5598f1d5ff360afc15c0", "LaunchPipelineReexecution": "d21e4ecaf3d1d163c4772f1d847dbdcbdaa9a40e6de0808a064ae767adf0c311", diff --git a/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx b/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx index ce8133c042eb1..291afd68962fc 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/app/Telemetry.tsx @@ -9,6 +9,7 @@ import {gql} from '../apollo-client'; export enum TelemetryAction { LAUNCH_RUN = 'LAUNCH_RUN', + LAUNCH_MULTIPLE_RUNS = 'LAUNCH_MULTIPLE_RUNS', GRAPHQL_QUERY_COMPLETED = 'GRAPHQL_QUERY_COMPLETED', } @@ -38,7 +39,7 @@ const LOG_TELEMETRY_MUTATION = gql` export async function logTelemetry( pathPrefix: string, action: TelemetryAction, - metadata: {[key: string]: string | null | undefined} = {}, + metadata: {[key: string]: string | string[] | null | undefined} = {}, ) { const graphqlPath = `${pathPrefix || ''}/graphql`; @@ -63,7 +64,10 @@ export async function logTelemetry( export const useTelemetryAction = () => { const {basePath, telemetryEnabled} = useContext(AppContext); return useCallback( - (action: TelemetryAction, metadata: {[key: string]: string | null | undefined} = {}) => { + ( + action: TelemetryAction, + metadata: {[key: string]: string | string[] | null | undefined} = {}, + ) => { if (telemetryEnabled) { logTelemetry(basePath, action, metadata); } diff --git a/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts new file mode 100644 index 0000000000000..f79ab12cea078 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-core/src/launchpad/useLaunchMultipleRunsWithTelemetry.ts @@ -0,0 +1,62 @@ +import {useCallback} from 'react'; +import {useHistory} from 'react-router-dom'; + +import {showLaunchError} from './showLaunchError'; +import {useMutation} from '../apollo-client'; +import {TelemetryAction, useTelemetryAction} from '../app/Telemetry'; +import { + LAUNCH_MULTIPLE_RUNS_MUTATION, + LaunchBehavior, + handleLaunchMultipleResult, +} from '../runs/RunUtils'; +import { + LaunchMultipleRunsMutation, + LaunchMultipleRunsMutationVariables, +} from '../runs/types/RunUtils.types'; + +export function useLaunchMultipleRunsWithTelemetry() { + const [launchMultipleRuns] = useMutation< + LaunchMultipleRunsMutation, + LaunchMultipleRunsMutationVariables + >(LAUNCH_MULTIPLE_RUNS_MUTATION); + + const logTelemetry = useTelemetryAction(); + const history = useHistory(); + + return useCallback( + async (variables: LaunchMultipleRunsMutationVariables, behavior: LaunchBehavior) => { + const executionParamsList = Array.isArray(variables.executionParamsList) + ? variables.executionParamsList + : [variables.executionParamsList]; + const jobNames = executionParamsList.map((params) => params.selector?.jobName); + + if (jobNames.length !== executionParamsList.length || jobNames.includes(undefined)) { + return; + } + + const metadata: {[key: string]: string | string[] | null | undefined} = { + jobNames: jobNames.filter((name): name is string => name !== undefined), + opSelection: undefined, + }; + + let result; + try { + result = (await launchMultipleRuns({variables})).data?.launchMultipleRuns; + if (result) { + handleLaunchMultipleResult(result, history, {behavior}); + logTelemetry( + TelemetryAction.LAUNCH_MULTIPLE_RUNS, + metadata as {[key: string]: string | string[] | null | undefined}, + ); + } + + return result; + } catch (error) { + console.error('error', error); + showLaunchError(error as Error); + } + return undefined; + }, + [history, launchMultipleRuns, logTelemetry], + ); +} diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx index 0c8115d91aa30..6a17b5cfc530c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunUtils.tsx @@ -7,7 +7,11 @@ import {StepSelection} from './StepSelection'; import {TimeElapsed} from './TimeElapsed'; import {RunFragment} from './types/RunFragments.types'; import {RunTableRunFragment} from './types/RunTableRunFragment.types'; -import {LaunchPipelineExecutionMutation, RunTimeFragment} from './types/RunUtils.types'; +import { + LaunchMultipleRunsMutation, + LaunchPipelineExecutionMutation, + RunTimeFragment, +} from './types/RunUtils.types'; import {Mono} from '../../../ui-components/src'; import {gql} from '../apollo-client'; import {showCustomAlert} from '../app/CustomAlertProvider'; @@ -107,7 +111,7 @@ export async function handleLaunchResult( if ('errors' in result) { message += ` Please fix the following errors:\n\n${result.errors - .map((error) => error.message) + .map((error: {message: any}) => error.message) .join('\n\n')}`; } @@ -115,6 +119,91 @@ export async function handleLaunchResult( } } +export async function handleLaunchMultipleResult( + result: void | null | LaunchMultipleRunsMutation['launchMultipleRuns'], + history: History, + options: {behavior: LaunchBehavior; preserveQuerystring?: boolean}, +) { + if (!result) { + showCustomAlert({body: `No data was returned. Did dagster-webserver crash?`}); + return; + } + const successfulRunIds: string[] = []; + const failedRunsErrors: {message: string}[] = []; + + if (result.__typename === 'PythonError') { + // if launch multiple runs errors out, show the PythonError and return + showCustomAlert({ + title: 'Error', + body: , + }); + return; + } else if (result.__typename === 'LaunchMultipleRunsResult') { + // show corresponding toasts + const launchMultipleRunsResult = result.launchMultipleRunsResult; + + for (const individualResult of launchMultipleRunsResult) { + if (individualResult.__typename === 'LaunchRunSuccess') { + successfulRunIds.push(individualResult.run.id); + + const pathname = `/runs/${individualResult.run.id}`; + const search = options.preserveQuerystring ? history.location.search : ''; + const openInSameTab = () => history.push({pathname, search}); + + // using open with multiple runs will spam new tabs + if (options.behavior === 'open') { + openInSameTab(); + } + } else if (individualResult.__typename === 'PythonError') { + failedRunsErrors.push({message: individualResult.message}); + } else { + let message = `Error launching run.`; + if ( + individualResult && + typeof individualResult === 'object' && + 'errors' in individualResult + ) { + const errors = individualResult.errors as {message: string}[]; + message += ` Please fix the following errors:\n\n${errors + .map((error) => error.message) + .join('\n\n')}`; + } + if ( + individualResult && + typeof individualResult === 'object' && + 'message' in individualResult + ) { + message += `\n\n${individualResult.message}`; + } + + failedRunsErrors.push({message}); + } + } + } + document.dispatchEvent(new CustomEvent('run-launched')); + + // link to runs page filtered to run IDs + const params = new URLSearchParams(); + successfulRunIds.forEach((id) => params.append('q[]', `id:${id}`)); + + const queryString = `/runs?${params.toString()}`; + history.push(queryString); + + await showSharedToaster({ + intent: 'success', + message:
Launched {successfulRunIds.length} runs
, + action: { + text: 'View', + href: history.createHref({pathname: queryString}), + }, + }); + + // show list of errors that occurred + if (failedRunsErrors.length > 0) { + showCustomAlert({body: failedRunsErrors.map((e) => e.message).join('\n\n')}); + } +} + function getBaseExecutionMetadata(run: RunFragment | RunTableRunFragment) { const hiddenTagKeys: string[] = [DagsterTag.IsResumeRetry, DagsterTag.StepSelection]; @@ -204,6 +293,65 @@ export const LAUNCH_PIPELINE_EXECUTION_MUTATION = gql` ${PYTHON_ERROR_FRAGMENT} `; +export const LAUNCH_MULTIPLE_RUNS_MUTATION = gql` + mutation LaunchMultipleRuns($executionParamsList: [ExecutionParams!]!) { + launchMultipleRuns(executionParamsList: $executionParamsList) { + __typename + ... on LaunchMultipleRunsResult { + launchMultipleRunsResult { + __typename + ... on InvalidStepError { + invalidStepKey + } + ... on InvalidOutputError { + stepKey + invalidOutputName + } + ... on LaunchRunSuccess { + run { + id + pipeline { + name + } + tags { + key + value + } + status + runConfigYaml + mode + resolvedOpSelection + } + } + ... on ConflictingExecutionParamsError { + message + } + ... on PresetNotFoundError { + preset + message + } + ... on RunConfigValidationInvalid { + pipelineName + errors { + __typename + message + path + reason + } + } + ... on PipelineNotFoundError { + message + pipelineName + } + ...PythonErrorFragment + } + } + ...PythonErrorFragment + } + } + ${PYTHON_ERROR_FRAGMENT} +`; + export const DELETE_MUTATION = gql` mutation Delete($runId: String!) { deletePipelineRun(runId: $runId) { diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts index f9dbe944212ae..0ce96156d07a9 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/types/RunUtils.types.ts @@ -42,6 +42,106 @@ export type LaunchPipelineExecutionMutation = { | {__typename: 'UnauthorizedError'}; }; +export type LaunchMultipleRunsMutationVariables = Types.Exact<{ + executionParamsList: Array | Types.ExecutionParams; +}>; + +export type LaunchMultipleRunsMutation = { + __typename: 'Mutation'; + launchMultipleRuns: + | { + __typename: 'LaunchMultipleRunsResult'; + launchMultipleRunsResult: Array< + | {__typename: 'ConflictingExecutionParamsError'; message: string} + | {__typename: 'InvalidOutputError'; stepKey: string; invalidOutputName: string} + | {__typename: 'InvalidStepError'; invalidStepKey: string} + | {__typename: 'InvalidSubsetError'} + | { + __typename: 'LaunchRunSuccess'; + run: { + __typename: 'Run'; + id: string; + status: Types.RunStatus; + runConfigYaml: string; + mode: string; + resolvedOpSelection: Array | null; + pipeline: + | {__typename: 'PipelineSnapshot'; name: string} + | {__typename: 'UnknownPipeline'; name: string}; + tags: Array<{__typename: 'PipelineTag'; key: string; value: string}>; + }; + } + | {__typename: 'NoModeProvidedError'} + | {__typename: 'PipelineNotFoundError'; message: string; pipelineName: string} + | {__typename: 'PresetNotFoundError'; preset: string; message: string} + | { + __typename: 'PythonError'; + message: string; + stack: Array; + errorChain: Array<{ + __typename: 'ErrorChainLink'; + isExplicitLink: boolean; + error: {__typename: 'PythonError'; message: string; stack: Array}; + }>; + } + | { + __typename: 'RunConfigValidationInvalid'; + pipelineName: string; + errors: Array< + | { + __typename: 'FieldNotDefinedConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'FieldsNotDefinedConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'MissingFieldConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'MissingFieldsConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'RuntimeMismatchConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + | { + __typename: 'SelectorTypeConfigError'; + message: string; + path: Array; + reason: Types.EvaluationErrorReason; + } + >; + } + | {__typename: 'RunConflict'} + | {__typename: 'UnauthorizedError'} + >; + } + | { + __typename: 'PythonError'; + message: string; + stack: Array; + errorChain: Array<{ + __typename: 'ErrorChainLink'; + isExplicitLink: boolean; + error: {__typename: 'PythonError'; message: string; stack: Array}; + }>; + }; +}; + export type DeleteMutationVariables = Types.Exact<{ runId: Types.Scalars['String']['input']; }>; @@ -168,6 +268,8 @@ export type RunTimeFragment = { export const LaunchPipelineExecutionVersion = '292088c4a697aca6be1d3bbc0cfc45d8a13cdb2e75cfedc64b68c6245ea34f89'; +export const LaunchMultipleRunsVersion = 'a56d9efdb35e71e0fd1744dd768129248943bc5b23e717458b82c46829661763'; + export const DeleteVersion = '3c61c79b99122910e754a8863e80dc5ed479a0c23cc1a9d9878d91e603fc0dfe'; export const TerminateVersion = '67acf403eb320a93c9a9aa07f675a1557e0887d499cd5598f1d5ff360afc15c0'; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index 0af78ed4cea37..533bc3e7c1cf1 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -13,6 +13,7 @@ import { Subheading, Tag, TextInput, + Tooltip, } from '@dagster-io/ui-components'; import {useCallback, useMemo, useState} from 'react'; import styled from 'styled-components'; @@ -31,15 +32,18 @@ import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorFragment'; import {PythonErrorInfo} from '../app/PythonErrorInfo'; import {assertUnreachable} from '../app/Util'; import {PythonErrorFragment} from '../app/types/PythonErrorFragment.types'; +import {SensorSelector} from '../graphql/types'; +import {useLaunchMultipleRunsWithTelemetry} from '../launchpad/useLaunchMultipleRunsWithTelemetry'; import {SET_CURSOR_MUTATION} from '../sensors/EditCursorDialog'; import { SetSensorCursorMutation, SetSensorCursorMutationVariables, } from '../sensors/types/EditCursorDialog.types'; import {testId} from '../testing/testId'; +import {buildExecutionParamsListSensor} from '../util/buildExecutionParamsList'; import {RepoAddress} from '../workspace/types'; -type DryRunInstigationTick = Extract< +export type SensorDryRunInstigationTick = Extract< SensorDryRunMutation['sensorDryRun'], {__typename: 'DryRunInstigationTick'} >; @@ -76,12 +80,12 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop const [cursor, setCursor] = useState(currentCursor); const [submitting, setSubmitting] = useState(false); + const [launching, setLaunching] = useState(false); const [error, setError] = useState(null); - const [sensorExecutionData, setSensorExecutionData] = useState( - null, - ); + const [sensorExecutionData, setSensorExecutionData] = + useState(null); - const sensorSelector = useMemo( + const sensorSelector: SensorSelector = useMemo( () => ({ sensorName: name, repositoryLocationName: repoAddress.location, @@ -90,6 +94,14 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop [repoAddress, name], ); + const executionParamsList = useMemo( + () => + sensorExecutionData && sensorSelector + ? buildExecutionParamsListSensor(sensorExecutionData, sensorSelector) + : [], + [sensorSelector, sensorExecutionData], + ); + const submitTest = useCallback(async () => { setSubmitting(true); const result = await sensorDryRun({ @@ -120,11 +132,47 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop setSubmitting(false); }, [sensorDryRun, sensorSelector, cursor, name]); + const launchMultipleRunsWithTelemetry = useLaunchMultipleRunsWithTelemetry(); + + const canLaunchAll = useMemo(() => { + return executionParamsList != null && executionParamsList.length > 0; + }, [executionParamsList]); + + const onLaunchAll = useCallback(async () => { + if (!canLaunchAll) { + return; + } + setLaunching(true); + + try { + if (executionParamsList) { + await launchMultipleRunsWithTelemetry({executionParamsList}, 'toast'); + } + } catch (e) { + console.error(e); + } + + setLaunching(false); + onClose(); + }, [canLaunchAll, executionParamsList, launchMultipleRunsWithTelemetry, onClose]); + const buttons = useMemo(() => { + if (launching) { + return ; + } + if (sensorExecutionData || error) { return ( - + + + + + + + ); } - if (data.scheduleOrError.__typename === 'PythonError') { - return
; - } - if (data.scheduleOrError.__typename === 'ScheduleNotFoundError') { - return
; - } - const timestamps = data.scheduleOrError.potentialTickTimestamps.map((ts) => ({ - ts, - label: timestampToString({ - timestamp: {unix: ts}, - locale, - timezone: userTimezone, - timeFormat: { - showTimezone: true, - }, - }), - })); - selectedTimestampRef.current = _selectedTimestamp || timestamps[0] || null; - return ( -
- Select a mock evaluation time - - {timestamps.map((timestamp) => ( - {timestamp.label}
} - onClick={() => { - setSelectedTimestamp(timestamp); - setIsTickSelectionOpen(false); - }} - /> - ))} - - } - > -
- -
- -
- ); }, [ - _selectedTimestamp, - containerProps, - data, - isTickSelectionOpen, - jobName, - name, + launching, + getScheduleData, + scheduleExecutionData, + scheduleExecutionError, + scheduleDryRunMutationLoading, repoAddress, - shouldEvaluate, - userTimezone, + name, + jobName, + selectedTimestamp, + isTickSelectionOpen, viewport.width, + containerProps, + userTimezone, ]); const buttons = useMemo(() => { - if (!shouldEvaluate) { + if (launching) { + return ; + } + + if (scheduleExecutionData || scheduleExecutionError) { + return ( + + + + + + + + + ); + } + + if (scheduleDryRunMutationLoading) { + return ( + + + + ); + } else { return ( <> ); - } else { - return ( - <> - - - - ); } - }, [onClose, shouldEvaluate]); + }, [ + canLaunchAll, + canSubmitTest, + launching, + onClose, + onLaunchAll, + scheduleExecutionData, + scheduleExecutionError, + submitTest, + scheduleDryRunMutationLoading, + ]); return ( <> @@ -226,62 +421,34 @@ export const GET_SCHEDULE_QUERY = gql` } `; -const EvaluateScheduleContent = ({ +// FE for showing result of evaluating schedule (error, skipped, or success state) +const EvaluateScheduleResult = ({ repoAddress, name, timestamp, jobName, + scheduleExecutionData, + scheduleExecutionError, }: { repoAddress: RepoAddress; name: string; timestamp: number; jobName: string; + scheduleExecutionData: ScheduleDryRunInstigationTick | null; + scheduleExecutionError: PythonErrorFragment | null; }) => { const { timezone: [userTimezone], } = useContext(TimeContext); - const [scheduleDryRunMutation] = useMutation< - ScheduleDryRunMutation, - ScheduleDryRunMutationVariables - >( - SCHEDULE_DRY_RUN_MUTATION, - useMemo(() => { - const repositorySelector = repoAddressToSelector(repoAddress); - return { - variables: { - selectorData: { - ...repositorySelector, - scheduleName: name, - }, - timestamp, - }, - }; - }, [name, repoAddress, timestamp]), - ); - const [result, setResult] = useState> | null>( - null, - ); - useEffect(() => { - scheduleDryRunMutation().then((result) => { - setResult(() => result); - }); - }, [scheduleDryRunMutation]); - - if (!result || !result.data) { - return ( - - - - ); - } - const evaluationResult = - result?.data?.scheduleDryRun.__typename === 'DryRunInstigationTick' - ? result?.data?.scheduleDryRun.evaluationResult - : null; + const evaluationResult = scheduleExecutionData?.evaluationResult; const innerContent = () => { - const data = result.data; + if (scheduleExecutionError) { + return ; + } + + const data = scheduleExecutionData; if (!data || !evaluationResult) { return ( @@ -363,6 +530,7 @@ const EvaluateScheduleContent = ({ ); }; + export const SCHEDULE_DRY_RUN_MUTATION = gql` mutation ScheduleDryRunMutation($selectorData: ScheduleSelector!, $timestamp: Float) { scheduleDryRun(selectorData: $selectorData, timestamp: $timestamp) { diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx index 03ea49e3fdf2d..1929282a95139 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx @@ -27,7 +27,6 @@ export const EvaluateTickButtonSchedule = ({ Evaluate tick { setShowTestTickDialog(false); diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx index c7970d1fba44c..c9406fcfc43f4 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/EvaluateScheduleDialog.test.tsx @@ -61,6 +61,9 @@ describe('EvaluateScheduleTest', () => { expect(screen.getByTestId('tick-5')).toBeVisible(); }); await userEvent.click(screen.getByTestId('tick-5')); + await waitFor(() => { + expect(screen.getByTestId('evaluate')).not.toBeDisabled(); + }); await userEvent.click(screen.getByTestId('evaluate')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); diff --git a/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts index b58e58c7b7611..d5264bf96a6b6 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/util/buildExecutionParamsList.ts @@ -1,8 +1,9 @@ import * as yaml from 'yaml'; import {showCustomAlert} from '../app/CustomAlertProvider'; -import {ExecutionParams, SensorSelector} from '../graphql/types'; +import {ExecutionParams, ScheduleSelector, SensorSelector} from '../graphql/types'; import {sanitizeConfigYamlString} from '../launchpad/yamlUtils'; +import {ScheduleDryRunInstigationTick} from '../ticks/EvaluateScheduleDialog'; import {SensorDryRunInstigationTick} from '../ticks/SensorDryRunDialog'; const YAML_SYNTAX_INVALID = `The YAML you provided couldn't be parsed. Please fix the syntax errors and try again.`; @@ -51,3 +52,45 @@ export const buildExecutionParamsListSensor = ( }); return executionParamsList; }; + +// adapted from buildExecutionVariables() in LaunchpadSession.tsx +export const buildExecutionParamsListSchedule = ( + scheduleExecutionData: ScheduleDryRunInstigationTick, + scheduleSelector: ScheduleSelector, +) => { + if (!scheduleExecutionData) { + return []; + } + + const executionParamsList: ExecutionParams[] = []; + + scheduleExecutionData?.evaluationResult?.runRequests?.forEach((request) => { + const configYamlOrEmpty = sanitizeConfigYamlString(request.runConfigYaml); + + try { + yaml.parse(configYamlOrEmpty); + } catch { + showCustomAlert({title: 'Invalid YAML', body: YAML_SYNTAX_INVALID}); + return; + } + const {repositoryLocationName, repositoryName} = scheduleSelector; + + const executionParams: ExecutionParams = { + runConfigData: configYamlOrEmpty, + selector: { + jobName: request.jobName, // get jobName from runRequest + repositoryLocationName, + repositoryName, + assetSelection: [], + assetCheckSelection: [], + solidSelection: undefined, + }, + mode: 'default', + executionMetadata: { + tags: [...request.tags.map(onlyKeyAndValue)], + }, + }; + executionParamsList.push(executionParams); + }); + return executionParamsList; +}; From 0286e4ab61fda6e1264375885236f4fa4d3c2274 Mon Sep 17 00:00:00 2001 From: David Liu Date: Fri, 22 Nov 2024 13:06:54 -0500 Subject: [PATCH 07/10] initial design tweaks --- .../VirtualizedAutomationScheduleRow.tsx | 15 ------------- .../VirtualizedAutomationSensorRow.tsx | 21 ------------------- .../src/ticks/EvaluateTickButtonSchedule.tsx | 2 +- .../src/ticks/EvaluateTickButtonSensor.tsx | 2 +- 4 files changed, 2 insertions(+), 38 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx index e4a8568d1f974..3132b7f265084 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationScheduleRow.tsx @@ -8,7 +8,6 @@ import { } from '@dagster-io/ui-components'; import {forwardRef, useMemo} from 'react'; import {Link} from 'react-router-dom'; -import styled from 'styled-components'; import {AutomationTargetList} from './AutomationTargetList'; import {AutomationRowGrid} from './VirtualizedAutomationRow'; @@ -25,7 +24,6 @@ import { ScheduleAssetSelectionQuery, ScheduleAssetSelectionQueryVariables, } from '../schedules/types/ScheduleAssetSelectionsQuery.types'; -import {EvaluateTickButtonSchedule} from '../ticks/EvaluateTickButtonSchedule'; import {TickStatusTag} from '../ticks/TickStatusTag'; import {RowCell} from '../ui/VirtualizedTable'; import {SINGLE_SCHEDULE_QUERY} from '../workspace/VirtualizedScheduleRow'; @@ -164,13 +162,6 @@ export const VirtualizedAutomationScheduleRow = forwardRef( - - - @@ -242,9 +233,3 @@ export const VirtualizedAutomationScheduleRow = forwardRef( ); }, ); - -const EvaluateTickButtonScheduleWrapper = styled.div` - button { - height: 24px; - } -`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx index 599551ffca9e5..1d37a69a14f41 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx @@ -8,7 +8,6 @@ import { } from '@dagster-io/ui-components'; import {forwardRef, useMemo} from 'react'; import {Link} from 'react-router-dom'; -import styled from 'styled-components'; import {AutomationTargetList} from './AutomationTargetList'; import {AutomationRowGrid} from './VirtualizedAutomationRow'; @@ -22,7 +21,6 @@ import { SensorAssetSelectionQuery, SensorAssetSelectionQueryVariables, } from '../sensors/types/SensorRoot.types'; -import {EvaluateTickButtonSensor} from '../ticks/EvaluateTickButtonSensor'; import {TickStatusTag} from '../ticks/TickStatusTag'; import {RowCell} from '../ui/VirtualizedTable'; import {SENSOR_TYPE_META, SINGLE_SENSOR_QUERY} from '../workspace/VirtualizedSensorRow'; @@ -163,19 +161,6 @@ export const VirtualizedAutomationSensorRow = forwardRef( - {sensorData ? ( - - - - ) : ( -
- )} @@ -235,9 +220,3 @@ export const VirtualizedAutomationSensorRow = forwardRef( ); }, ); - -const EvaluateTickButtonSensorWrapper = styled.div` - button { - height: 24px; - } -`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx index 1929282a95139..a579e66913f41 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx @@ -24,7 +24,7 @@ export const EvaluateTickButtonSchedule = ({ setShowTestTickDialog(true); }} > - Evaluate tick + Preview tick result setShowTestTickDialog(true)} > - Evaluate tick + Preview tick result Date: Mon, 25 Nov 2024 12:42:07 -0500 Subject: [PATCH 08/10] design tweaks for cursor page --- .../ui-components/src/components/Icon.tsx | 2 + .../src/icon-svgs/preview_tick.svg | 4 ++ .../src/ticks/EvaluateTickButtonSensor.tsx | 3 +- .../ui-core/src/ticks/SensorDryRunDialog.tsx | 40 ++++++++----------- .../__tests__/SensorDryRunDialog.test.tsx | 8 ++-- 5 files changed, 29 insertions(+), 28 deletions(-) create mode 100644 js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg diff --git a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx index b5e4763261e91..5cb21262773f9 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx +++ b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx @@ -268,6 +268,7 @@ import password from '../icon-svgs/password.svg'; import pause from '../icon-svgs/pause.svg'; import people from '../icon-svgs/people.svg'; import plots from '../icon-svgs/plots.svg'; +import preview_tick from '../icon-svgs/preview_tick.svg'; import priority_1 from '../icon-svgs/priority_1.svg'; import priority_2 from '../icon-svgs/priority_2.svg'; import priority_3 from '../icon-svgs/priority_3.svg'; @@ -664,6 +665,7 @@ export const Icons = { pause, people, plots, + preview_tick, priority_1, priority_2, priority_3, diff --git a/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg new file mode 100644 index 0000000000000..1f6680536c263 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/preview_tick.svg @@ -0,0 +1,4 @@ + + + + diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx index 5492b2f98cd88..dddbe42c20c50 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSensor.tsx @@ -1,4 +1,4 @@ -import {Box, Button, Tooltip} from '@dagster-io/ui-components'; +import {Box, Button, Icon, Tooltip} from '@dagster-io/ui-components'; import {useState} from 'react'; import {SensorDryRunDialog} from './SensorDryRunDialog'; @@ -32,6 +32,7 @@ export const EvaluateTickButtonSensor = ({ diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx index 533bc3e7c1cf1..619b1ff233461 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/SensorDryRunDialog.tsx @@ -8,7 +8,6 @@ import { DialogFooter, Group, Icon, - NonIdealState, Spinner, Subheading, Tag, @@ -64,8 +63,8 @@ export const SensorDryRunDialog = (props: Props) => { isOpen={isOpen} onClose={onClose} style={{width: '70vw', display: 'flex'}} - icon="sensors" - title={name} + icon="preview_tick" + title={`Preview tick result for ${name}`} > @@ -198,8 +197,8 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop return ( - ); @@ -376,29 +375,24 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop } else { return ( -
Cursor
+
Cursor value (optional)
setCursor(e.target.value)} data-testid={testId('cursor-input')} + placeholder="Enter a cursor value" /> - {currentCursor === '' || !currentCursor ? ( - - - Check our{' '} - - sensor documentation - {' '} - to learn how to use cursors - - } - /> - - ) : null} +
+ A cursor tracks where a sensor left off, allowing the sensor to efficiently process new + changes or events without missing anything or duplicating work. The cursor is typically + a string, and can be updated within the sensor's logic to reflect the latest state. +
+
+ + Learn more + {' '} + about cursors +
); } diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx index 5972b015f2f43..8bbf5690f0345 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx @@ -38,7 +38,7 @@ describe('SensorDryRunTest', () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText(/3\srun requests/g)).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); @@ -55,7 +55,7 @@ describe('SensorDryRunTest', () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); @@ -66,7 +66,7 @@ describe('SensorDryRunTest', () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); @@ -81,7 +81,7 @@ describe('SensorDryRunTest', () => { render(); const cursorInput = await screen.findByTestId('cursor-input'); await userEvent.type(cursorInput, 'testing123'); - await userEvent.click(screen.getByTestId('evaluate')); + await userEvent.click(screen.getByTestId('continue')); await waitFor(() => { expect(screen.getByText('Skipped')).toBeVisible(); }); From d4b36696213233a10183159347c89a90deea9225 Mon Sep 17 00:00:00 2001 From: David Liu Date: Thu, 5 Dec 2024 16:49:06 -0500 Subject: [PATCH 09/10] sensor result page redesign --- .../ui-components/src/components/Icon.tsx | 2 + .../src/components/NonIdealState.tsx | 7 +- .../src/icon-svgs/data_object.svg | 1 + .../ui-core/src/runs/RunConfigDialog.tsx | 73 +++- .../ui-core/src/ticks/DryRunRequestTable.tsx | 114 ++---- .../ui-core/src/ticks/SensorDryRunDialog.tsx | 378 ++++++++++-------- .../__tests__/SensorDryRunDialog.test.tsx | 7 +- 7 files changed, 345 insertions(+), 237 deletions(-) create mode 100644 js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg diff --git a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx index 5cb21262773f9..e2326328b9cd2 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx +++ b/js_modules/dagster-ui/packages/ui-components/src/components/Icon.tsx @@ -101,6 +101,7 @@ import dagster_reversed from '../icon-svgs/dagster_reversed.svg'; import dagster_solid from '../icon-svgs/dagster_solid.svg'; import dagsterlabs from '../icon-svgs/dagsterlabs.svg'; import dash from '../icon-svgs/dash.svg'; +import data_object from '../icon-svgs/data_object.svg'; import data_reliability from '../icon-svgs/data_reliability.svg'; import data_type from '../icon-svgs/data_type.svg'; import database from '../icon-svgs/database.svg'; @@ -503,6 +504,7 @@ export const Icons = { dash, data_reliability, data_type, + data_object, database, datatype_array, datatype_bool, diff --git a/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx b/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx index 6931dabab7d1e..037f355b399ce 100644 --- a/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx +++ b/js_modules/dagster-ui/packages/ui-components/src/components/NonIdealState.tsx @@ -1,4 +1,5 @@ import * as React from 'react'; +import styled from 'styled-components'; import {Box} from './Box'; import {Colors} from './Color'; @@ -27,7 +28,7 @@ export const NonIdealState = ({ const singleContentElement = [title, description, action].filter(Boolean).length === 1; return ( - {description}
} {action} - + ); }; + +export const NonIdealStateWrapper = styled(Box)``; diff --git a/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg new file mode 100644 index 0000000000000..8199c504d3bf9 --- /dev/null +++ b/js_modules/dagster-ui/packages/ui-components/src/icon-svgs/data_object.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx index 3ad2312952379..a302013baefb6 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/runs/RunConfigDialog.tsx @@ -3,6 +3,7 @@ import { Button, Dialog, DialogFooter, + Icon, StyledRawCodeMirror, Subheading, } from '@dagster-io/ui-components'; @@ -10,6 +11,11 @@ import styled from 'styled-components'; import {RunTags} from './RunTags'; import {RunTagsFragment} from './types/RunTagsFragment.types'; +import {applyCreateSession, useExecutionSessionStorage} from '../app/ExecutionSessionStorage'; +import {useOpenInNewTab} from '../hooks/useOpenInNewTab'; +import {RunRequestFragment} from '../ticks/types/RunRequestFragment.types'; +import {RepoAddress} from '../workspace/types'; +import {workspacePathFromAddress} from '../workspace/workspacePath'; interface Props { isOpen: boolean; @@ -21,10 +27,15 @@ interface Props { // Optionally provide tags to display them as well. tags?: RunTagsFragment[]; + + // Optionally provide a request to display the "Open in Launchpad" button. + request?: RunRequestFragment; + repoAddress?: RepoAddress; } export const RunConfigDialog = (props: Props) => { - const {isOpen, onClose, copyConfig, runConfigYaml, tags, mode, isJob} = props; + const {isOpen, onClose, copyConfig, runConfigYaml, tags, mode, isJob, request, repoAddress} = + props; const hasTags = !!tags && tags.length > 0; return ( @@ -68,7 +79,20 @@ export const RunConfigDialog = (props: Props) => { - + + ) + } + > @@ -81,6 +105,51 @@ export const RunConfigDialog = (props: Props) => { ); }; +function OpenInLaunchpadButton({ + mode, + request, + jobName, + isJob, + repoAddress, +}: { + request: RunRequestFragment; + jobName?: string; + mode?: string | null; + repoAddress: RepoAddress; + isJob: boolean; +}) { + const openInNewTab = useOpenInNewTab(); + const pipelineName = request.jobName ?? jobName; + const [_, onSave] = useExecutionSessionStorage(repoAddress, pipelineName!); + + return ( + + ); +} + const CodeMirrorContainer = styled.div` flex: 1; overflow: hidden; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx index 492f26f1543f6..ca8082e8561fb 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/DryRunRequestTable.tsx @@ -1,13 +1,14 @@ -import {Box, Button, Colors, Icon, Table, Tag} from '@dagster-io/ui-components'; +import {Box, Button, Colors, Icon, Table, Tooltip} from '@dagster-io/ui-components'; +import {useState} from 'react'; -import {applyCreateSession, useExecutionSessionStorage} from '../app/ExecutionSessionStorage'; +import {RunConfigDialog} from '../runs/RunConfigDialog'; import {RunRequestFragment} from './types/RunRequestFragment.types'; -import {useOpenInNewTab} from '../hooks/useOpenInNewTab'; +import {showSharedToaster} from '../app/DomUtils'; +import {useCopyToClipboard} from '../app/browser'; import {PipelineReference} from '../pipelines/PipelineReference'; import {testId} from '../testing/testId'; import {useRepository} from '../workspace/WorkspaceContext/util'; import {RepoAddress} from '../workspace/types'; -import {workspacePathFromAddress} from '../workspace/workspacePath'; type Props = { name: string; @@ -20,13 +21,25 @@ type Props = { export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName}: Props) => { const repo = useRepository(repoAddress); + const [selectedRequest, setSelectedRequest] = useState(null); + const [visibleDialog, setVisibleDialog] = useState<'config' | null>(null); + const copy = useCopyToClipboard(); + + const copyConfig = async () => { + copy(selectedRequest?.runConfigYaml || ''); + await showSharedToaster({ + intent: 'success', + icon: 'copy_to_clipboard_done', + message: 'Copied!', + }); + }; const body = ( {runRequests.map((request, index) => { return ( - + - - - {filterTags(request.tags).map(({key, value}) => ( - {`${key}: ${value}`} - ))} - - - - + { + setSelectedRequest(request); + setVisibleDialog('config'); + }} /> ); })} + {selectedRequest && ( + setVisibleDialog(null)} + copyConfig={() => copyConfig()} + mode={mode || null} + runConfigYaml={selectedRequest.runConfigYaml} + tags={selectedRequest.tags} + isJob={isJob} + request={selectedRequest} + repoAddress={repoAddress} + /> + )} ); return ( @@ -63,9 +81,8 @@ export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName} - - - + + {body} @@ -74,55 +91,10 @@ export const RunRequestTable = ({runRequests, isJob, repoAddress, mode, jobName} ); }; -// Filter out tags we already display in other ways -function filterTags(tags: Array<{key: string; value: any}>) { - return tags.filter(({key}) => { - // Exclude the tag that specifies the schedule if this is a schedule name - return !['dagster/schedule_name'].includes(key); - }); -} - -function OpenInLaunchpadButton({ - mode, - request, - jobName, - isJob, - repoAddress, -}: { - request: RunRequestFragment; - jobName?: string; - mode?: string; - repoAddress: RepoAddress; - isJob: boolean; -}) { - const openInNewTab = useOpenInNewTab(); - const pipelineName = request.jobName ?? jobName; - const [_, onSave] = useExecutionSessionStorage(repoAddress, pipelineName!); - +function PreviewButton({onClick}: {onClick: () => void}) { return ( - + + - - - - + ); + } else { + return null; + } + }, [launching, sensorExecutionData, error]); + + const rightButtons = useMemo(() => { + if (launching) { + return ; + } + + if (sensorExecutionData || error) { + const runRequests = sensorExecutionData?.evaluationResult?.runRequests; + const numRunRequests = runRequests?.length || 0; + const didSkip = !error && numRunRequests === 0; + + if (error) { + return ( + + + + ); + } else if (didSkip) { + return ( + + + + + + + + ); + } else { + return ( + + + + + + + ); + } } if (submitting) { return ( @@ -208,62 +308,13 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop sensorExecutionData, error, submitting, + onClose, + onCommitTickResult, canLaunchAll, onLaunchAll, - onClose, submitTest, ]); - const [cursorState, setCursorState] = useState<'Unpersisted' | 'Persisting' | 'Persisted'>( - 'Unpersisted', - ); - const [setCursorMutation] = useMutation< - SetSensorCursorMutation, - SetSensorCursorMutationVariables - >(SET_CURSOR_MUTATION); - - const onPersistCursorValue = useCallback(async () => { - const cursor = sensorExecutionData?.evaluationResult?.cursor; - if (!cursor) { - assertUnreachable('Did not expect to get here' as never); - } - setCursorState('Persisting'); - const {data} = await setCursorMutation({ - variables: {sensorSelector, cursor}, - }); - if (data?.setSensorCursor.__typename === 'Sensor') { - await showSharedToaster({message: 'Cursor value updated', intent: 'success'}); - setCursorState('Persisted'); - } else if (data?.setSensorCursor) { - const error = data.setSensorCursor; - await showSharedToaster({ - intent: 'danger', - message: ( - -
Could not set cursor value.
- { - showCustomAlert({ - title: 'Python Error', - body: - error.__typename === 'PythonError' ? ( - - ) : ( - 'Sensor not found' - ), - }); - }} - > - View error - -
- ), - }); - } - }, [sensorExecutionData?.evaluationResult?.cursor, sensorSelector, setCursorMutation]); - const content = useMemo(() => { if (launching) { return ( @@ -281,80 +332,71 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop sensorExecutionData?.evaluationResult?.dynamicPartitionsRequests; return ( + +
+ Result + +
+ {error ? ( + Failed + ) : numRunRequests ? ( + {numRunRequests} run requests + ) : ( + Skipped + )} +
+
+
+
+ Used cursor value +
{cursor?.length ? cursor : 'None'}
+
+
- -
- Result - -
- {error ? ( - Failed - ) : numRunRequests ? ( - {numRunRequests} run requests - ) : ( - Skipped - )} -
-
-
-
- Used cursor value -
{cursor?.length ? cursor : 'None'}
-
-
- Computed cursor value -
-                  {sensorExecutionData?.evaluationResult?.cursor?.length
-                    ? sensorExecutionData?.evaluationResult.cursor
-                    : error
-                      ? 'Error'
-                      : 'None'}
-                
- {error || - (currentCursor ?? '') === - (sensorExecutionData?.evaluationResult?.cursor ?? '') ? null : ( - - - {cursorState === 'Persisted' ? ( - - ) : null} - - )} -
-
{error ? (
) : null} {didSkip ? ( -
- Skip reason + + Requested runs (0)
- {sensorExecutionData?.evaluationResult?.skipReason || 'No skip reason was output'} + + + + The sensor function was successfully evaluated but didn't return + any run requests. + + +
+ Skip reason:{' '} + {sensorExecutionData?.evaluationResult?.skipReason + ? `"${sensorExecutionData.evaluationResult.skipReason}"` + : 'No skip reason was output'} +
+ + } + /> +
-
+
) : null} {numRunRequests && runRequests ? ( - + + Requested runs ({numRunRequests}) + + ) : null} {dynamicPartitionRequests?.length ? (
@@ -362,6 +404,17 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop
) : null}
+ + + Computed cursor value +
+              {sensorExecutionData?.evaluationResult?.cursor?.length
+                ? sensorExecutionData?.evaluationResult.cursor
+                : error
+                  ? 'Error'
+                  : 'None'}
+            
+
); } @@ -396,26 +449,16 @@ const SensorDryRun = ({repoAddress, name, currentCursor, onClose, jobName}: Prop ); } - }, [ - sensorExecutionData, - error, - submitting, - launching, - currentCursor, - cursorState, - onPersistCursorValue, - name, - jobName, - repoAddress, - cursor, - ]); + }, [sensorExecutionData, error, submitting, launching, name, jobName, repoAddress, cursor]); return ( <>
{content}
- {buttons} + + {rightButtons} + ); }; @@ -455,10 +498,9 @@ export const EVALUATE_SENSOR_MUTATION = gql` const Grid = styled.div` display: grid; - grid-template-columns: repeat(3, 1fr); + grid-template-columns: repeat(2, 1fr); padding-bottom: 12px; border-bottom: 1px solid ${Colors.keylineDefault()}; - margin-bottom: 12px; ${Subheading} { padding-bottom: 4px; display: block; @@ -470,3 +512,27 @@ const Grid = styled.div` margin-top: 4px; } `; + +const ComputedCursorGrid = styled.div` + display: grid; + grid-template-columns: repeat(1, 1fr); + padding-bottom: 12px; + ${Subheading} { + padding-bottom: 4px; + display: block; + } + pre { + margin: 0; + } + button { + margin-top: 4px; + } +`; + +const SkipReasonNonIdealStateWrapper = styled.div` + ${NonIdealStateWrapper} { + margin: auto !important; + width: unset !important; + max-width: unset !important; + } +`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx index 8bbf5690f0345..4e23fef7c256c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/__tests__/SensorDryRunDialog.test.tsx @@ -44,11 +44,6 @@ describe('SensorDryRunTest', () => { expect(screen.queryByText('Skipped')).toBe(null); expect(screen.queryByText('Failed')).toBe(null); }); - await userEvent.click(screen.getByTestId('persist-cursor')); - expect(screen.getByText('Persisting')).toBeVisible(); - await waitFor(() => { - expect(screen.getByText('Persisted')).toBeVisible(); - }); }); it('renders errors', async () => { @@ -71,7 +66,7 @@ describe('SensorDryRunTest', () => { expect(screen.getByText('Failed')).toBeVisible(); expect(screen.queryByText('Skipped')).toBe(null); }); - await userEvent.click(screen.getByTestId('test-again')); + await userEvent.click(screen.getByTestId('try-again')); expect(screen.queryByText('Failed')).toBe(null); expect(screen.queryByText('Skipped')).toBe(null); expect(screen.getByTestId('cursor-input')).toBeVisible(); From d0aa12f4c9de16a216fdfa2850a476ed83579082 Mon Sep 17 00:00:00 2001 From: David Liu Date: Tue, 3 Dec 2024 15:12:30 -0500 Subject: [PATCH 10/10] redesign for schedules --- .../VirtualizedAutomationSensorRow.tsx | 6 - .../src/ticks/EvaluateScheduleDialog.tsx | 170 +++++++++++++----- .../src/ticks/EvaluateTickButtonSchedule.tsx | 3 +- .../__tests__/EvaluateScheduleDialog.test.tsx | 8 +- 4 files changed, 129 insertions(+), 58 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx index 1d37a69a14f41..19a80c0cd2196 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/automation/VirtualizedAutomationSensorRow.tsx @@ -88,12 +88,6 @@ export const VirtualizedAutomationSensorRow = forwardRef( return data.sensorOrError; }, [data]); - const cursor = - sensorData && - sensorData.sensorState.typeSpecificData && - sensorData.sensorState.typeSpecificData.__typename === 'SensorData' && - sensorData.sensorState.typeSpecificData.lastCursor; - const onChange = (e: React.FormEvent) => { if (onToggleChecked && e.target instanceof HTMLInputElement) { const {checked} = e.target; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx index ddfd6b42faba7..06add60352b2c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateScheduleDialog.tsx @@ -10,6 +10,7 @@ import { MenuItem, Mono, NonIdealState, + NonIdealStateWrapper, Popover, Spinner, Subheading, @@ -65,8 +66,8 @@ export const EvaluateScheduleDialog = (props: Props) => { style={{width: '70vw', display: 'flex'}} title={ - - {props.name} + + Preview tick result for {props.name} } > @@ -266,8 +267,8 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { })); selectedTimestampRef.current = selectedTimestamp || timestamps[0] || null; return ( -
- Select a mock evaluation time + + Select an evaluation time to simulate {
- +
+ Each evaluation of a schedule is called a tick, which is an opportunity for one or more + runs to be launched. Ticks kick off runs, which either materialize a selection of assets + or execute a job. + You can preview the result for a given tick in the next step. +
+
+ Learn more about + schedules +
+ ); } }, [ @@ -317,59 +328,88 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => { userTimezone, ]); - const buttons = useMemo(() => { + const leftButtons = useMemo(() => { if (launching) { - return ; + return null; } if (scheduleExecutionData || scheduleExecutionError) { return ( - - - - - - - - + ); + } else { + return null; + } + }, [launching, scheduleExecutionData, scheduleExecutionError]); + + const rightButtons = useMemo(() => { + if (launching) { + return ; + } + + if (scheduleExecutionData || scheduleExecutionError) { + const runRequests = scheduleExecutionData?.evaluationResult?.runRequests; + const numRunRequests = runRequests?.length || 0; + const didSkip = !scheduleExecutionError && numRunRequests === 0; + + if (scheduleExecutionError || didSkip) { + return ( + + + + ); + } else { + return ( + + + + + + + ); + } } if (scheduleDryRunMutationLoading) { return ( - + ); } else { return ( <> - + ); @@ -391,7 +431,9 @@ const EvaluateSchedule = ({repoAddress, name, onClose, jobName}: Props) => {
{content}
- {buttons ? {buttons} : null} + + {rightButtons} + ); }; @@ -449,6 +491,7 @@ const EvaluateScheduleResult = ({ } const data = scheduleExecutionData; + if (!data || !evaluationResult) { return ( - Skip Reason -
{evaluationResult?.skipReason || 'No skip reason was output'}
- + + Requested runs (0) +
+ + + + The schedule function was successfully evaluated but didn't return any + run requests. + + +
+ Skip reason:{' '} + {evaluationResult?.skipReason + ? `"${evaluationResult.skipReason}"` + : 'No skip reason was output'} +
+ + } + /> +
+
+
); } else { return ( - + + Requested runs ({numRunRequests}) + + ); } }; @@ -577,3 +645,11 @@ const Grid = styled.div` const ScheduleDescriptor = styled.div` padding-bottom: 2px; `; + +const SkipReasonNonIdealStateWrapper = styled.div` + ${NonIdealStateWrapper} { + margin: auto !important; + width: unset !important; + max-width: unset !important; + } +`; diff --git a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx index a579e66913f41..4713f65e5aa5d 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/ticks/EvaluateTickButtonSchedule.tsx @@ -1,4 +1,4 @@ -import {Box, Button} from '@dagster-io/ui-components'; +import {Box, Button, Icon} from '@dagster-io/ui-components'; import {useState} from 'react'; import {EvaluateScheduleDialog} from './EvaluateScheduleDialog'; @@ -20,6 +20,7 @@ export const EvaluateTickButtonSchedule = ({ return (
{isJob ? 'Job' : 'Pipeline'} nameTagsConfigurationTargetActions