From 71b739d5de8d7c397d4378d9467461733a7b990d Mon Sep 17 00:00:00 2001 From: Marijn Valk Date: Tue, 8 Oct 2024 22:26:03 +0200 Subject: [PATCH] Early check for invalid partition keys on launching a partition backfill (#25107) ## Summary & Motivation This adds an early check for on the partition keys that are supplied during a backfill mutation call. If an invalid partition key is found, a `PartitionKeysNotFoundError` is returned and the backfill is not scheduled for execution. Contributes to https://github.com/dagster-io/dagster/issues/23801 ## How I Tested These Changes Unit tests ## Changelog If a `LaunchPartitionBackfill` mutation is submitted to GQL with invalid partition keys, it will return an early `PartitionKeysNotFoundError`. - [x] `NEW` _(added new feature or capability)_ - [ ] `BUGFIX` _(fixed a bug)_ - [ ] `DOCS` _(added or updated documentation)_ --- .../dagster-ui/packages/ui-core/client.json | 2 +- .../src/graphql/possibleTypes.generated.json | 2 +- .../ui-core/src/graphql/schema.graphql | 6 ++ .../packages/ui-core/src/graphql/types.ts | 21 +++++++ .../src/instance/backfill/BackfillUtils.tsx | 3 + .../backfill/types/BackfillUtils.types.ts | 3 +- .../dagster_graphql/client/query.py | 3 + .../implementation/execution/backfill.py | 46 ++++++++++----- .../dagster_graphql/implementation/utils.py | 58 +++++++++++++++++++ .../dagster_graphql/schema/backfill.py | 2 + .../dagster_graphql/schema/errors.py | 17 +++++- .../graphql/test_asset_backfill.py | 27 +++++++++ .../graphql/test_partition_backfill.py | 25 ++++++++ 13 files changed, 197 insertions(+), 18 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/client.json b/js_modules/dagster-ui/packages/ui-core/client.json index 6b1862ce0eaf6..2e2581f46609a 100644 --- a/js_modules/dagster-ui/packages/ui-core/client.json +++ b/js_modules/dagster-ui/packages/ui-core/client.json @@ -67,7 +67,7 @@ "SingleBackfillQuery": "c2b27d4666926a1c0bfd0c7cfabf9840c67e33a0a374651ee8e52bdec395aa56", "CancelBackfill": "138f5ba5d38b0d939a6a0bf34769cf36c16bb99225204e28e5ab5fcd8baf3194", "resumeBackfill": "89327db4bc94c65ac484fa9b914f28b6c70f09dce91bbaab9aec8593188bdcbe", - "LaunchPartitionBackfill": "12402e3e19e32ede40dbc04673d452c2d4b3e6a09509c80fdd065c92e3640daa", + "LaunchPartitionBackfill": "07bf7523e7b8696598d621aad89a48f25e173a3955ab84dd60a745c21aff2d9b", "BackfillDetailsQuery": "b2e50b372c779a994a7fdcdbef873890bf39716520774315b9e880ae59375217", "InstanceConfigHasInfo": "771982a9ee439781255f82986d55aa6a75ab2929d784f2cd27b40f537baf7f27", "InstanceWarningQuery": "7ead177b08f678cb85bfebac63b1b25fff6c60c50c862a3c3d20d8d701463823", 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 c7cababb4ec26..6c623bc7d0789 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"],"ScheduleOrError":["Schedule","ScheduleNotFoundError","PythonError"],"SchedulerOrError":["Scheduler","SchedulerNotDefinedError","PythonError"],"SchedulesOrError":["Schedules","RepositoryNotFoundError","PythonError"],"ScheduleTickSpecificData":["ScheduleTickSuccessData","ScheduleTickFailureData"],"LaunchBackfillResult":["LaunchBackfillSuccess","PartitionSetNotFoundError","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","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"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourcesOrError":["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"],"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"],"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"],"GraphOrError":["Graph","GraphNotFoundError","PythonError"],"ResourceDetailsOrError":["ResourceDetails","ResourceNotFoundError","PythonError"],"ResourcesOrError":["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 1eea13085353f..db2e403ce645b 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 @@ -1719,6 +1719,7 @@ type AssetKey { union LaunchBackfillResult = LaunchBackfillSuccess | PartitionSetNotFoundError + | PartitionKeysNotFoundError | InvalidStepError | InvalidOutputError | RunConfigValidationInvalid @@ -1731,6 +1732,11 @@ union LaunchBackfillResult = | ConflictingExecutionParamsError | NoModeProvidedError +type PartitionKeysNotFoundError implements Error { + message: String! + partitionKeys: [String!]! +} + type LaunchBackfillSuccess { backfillId: String! launchedRunIds: [String] 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 234f635c8fddc..a18de1907bb53 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 @@ -2193,6 +2193,7 @@ export type LaunchBackfillResult = | InvalidSubsetError | LaunchBackfillSuccess | NoModeProvidedError + | PartitionKeysNotFoundError | PartitionSetNotFoundError | PipelineNotFoundError | PresetNotFoundError @@ -3163,6 +3164,12 @@ export type PartitionKeys = { partitionKeys: Array; }; +export type PartitionKeysNotFoundError = Error & { + __typename: 'PartitionKeysNotFoundError'; + message: Scalars['String']['output']; + partitionKeys: Array; +}; + export type PartitionKeysOrError = PartitionKeys | PartitionSubsetDeserializationError; export type PartitionMapping = { @@ -10950,6 +10957,20 @@ export const buildPartitionKeys = ( }; }; +export const buildPartitionKeysNotFoundError = ( + overrides?: Partial, + _relationshipsToOmit: Set = new Set(), +): {__typename: 'PartitionKeysNotFoundError'} & PartitionKeysNotFoundError => { + const relationshipsToOmit: Set = new Set(_relationshipsToOmit); + relationshipsToOmit.add('PartitionKeysNotFoundError'); + return { + __typename: 'PartitionKeysNotFoundError', + message: overrides && overrides.hasOwnProperty('message') ? overrides.message! : 'minima', + partitionKeys: + overrides && overrides.hasOwnProperty('partitionKeys') ? overrides.partitionKeys! : [], + }; +}; + export const buildPartitionMapping = ( overrides?: Partial, _relationshipsToOmit: Set = new Set(), diff --git a/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/BackfillUtils.tsx b/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/BackfillUtils.tsx index f23fcb8a6ec8a..1eb40e17ba03c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/BackfillUtils.tsx +++ b/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/BackfillUtils.tsx @@ -26,6 +26,9 @@ export const LAUNCH_PARTITION_BACKFILL_MUTATION = gql` ... on PartitionSetNotFoundError { message } + ... on PartitionKeysNotFoundError { + message + } ... on InvalidStepError { invalidStepKey } diff --git a/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/types/BackfillUtils.types.ts b/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/types/BackfillUtils.types.ts index 378cd7ddad62e..7934917165e7b 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/types/BackfillUtils.types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/instance/backfill/types/BackfillUtils.types.ts @@ -36,6 +36,7 @@ export type LaunchPartitionBackfillMutation = { | {__typename: 'InvalidSubsetError'} | {__typename: 'LaunchBackfillSuccess'; backfillId: string} | {__typename: 'NoModeProvidedError'} + | {__typename: 'PartitionKeysNotFoundError'; message: string} | {__typename: 'PartitionSetNotFoundError'; message: string} | {__typename: 'PipelineNotFoundError'; message: string} | {__typename: 'PresetNotFoundError'; message: string} @@ -95,4 +96,4 @@ export type LaunchPartitionBackfillMutation = { | {__typename: 'UnauthorizedError'; message: string}; }; -export const LaunchPartitionBackfillVersion = '12402e3e19e32ede40dbc04673d452c2d4b3e6a09509c80fdd065c92e3640daa'; +export const LaunchPartitionBackfillVersion = '07bf7523e7b8696598d621aad89a48f25e173a3955ab84dd60a745c21aff2d9b'; diff --git a/python_modules/dagster-graphql/dagster_graphql/client/query.py b/python_modules/dagster-graphql/dagster_graphql/client/query.py index 863e68998086b..ae00e1e5bd3f5 100644 --- a/python_modules/dagster-graphql/dagster_graphql/client/query.py +++ b/python_modules/dagster-graphql/dagster_graphql/client/query.py @@ -410,6 +410,9 @@ ... on PartitionSetNotFoundError { message } + ... on PartitionKeysNotFoundError { + message + } ... on LaunchBackfillSuccess { backfillId launchedRunIds diff --git a/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py b/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py index 95fbb112da65b..5b019ef536098 100644 --- a/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql/implementation/execution/backfill.py @@ -23,6 +23,8 @@ BackfillParams, assert_permission_for_asset_graph, assert_permission_for_location, + assert_valid_asset_partition_backfill, + assert_valid_job_partition_backfill, ) BACKFILL_CHUNK_SIZE = 25 @@ -85,6 +87,14 @@ def create_and_launch_partition_backfill( from dagster_graphql.schema.errors import GraphenePartitionSetNotFoundError backfill_id = make_new_backfill_id() + backfill_timestamp = get_current_timestamp() + backfill_datetime = datetime_from_timestamp(backfill_timestamp) + dynamic_partitions_store = CachingInstanceQueryer( + instance=graphene_info.context.instance, + asset_graph=graphene_info.context.asset_graph, + loading_context=graphene_info.context, + evaluation_time=backfill_datetime, + ) asset_selection = ( [ @@ -109,8 +119,6 @@ def create_and_launch_partition_backfill( tags = {**tags, **graphene_info.context.get_viewer_tags()} - backfill_timestamp = get_current_timestamp() - if backfill_params.get("selector") is not None: # job backfill partition_set_selector = backfill_params["selector"] partition_set_name = partition_set_selector.get("partitionSetName") @@ -172,6 +180,13 @@ def create_and_launch_partition_backfill( title=backfill_params.get("title"), description=backfill_params.get("description"), ) + assert_valid_job_partition_backfill( + graphene_info, + backfill, + external_partition_set.get_partitions_definition(), + dynamic_partitions_store, + backfill_datetime, + ) if backfill_params.get("forceSynchronousSubmission"): # should only be used in a test situation @@ -216,16 +231,17 @@ def create_and_launch_partition_backfill( backfill_timestamp=backfill_timestamp, asset_selection=asset_selection, partition_names=backfill_params.get("partitionNames"), - dynamic_partitions_store=CachingInstanceQueryer( - instance=graphene_info.context.instance, - asset_graph=asset_graph, - loading_context=graphene_info.context, - evaluation_time=datetime_from_timestamp(backfill_timestamp), - ), + dynamic_partitions_store=dynamic_partitions_store, all_partitions=backfill_params.get("allPartitions", False), title=backfill_params.get("title"), description=backfill_params.get("description"), ) + assert_valid_asset_partition_backfill( + graphene_info, + backfill, + dynamic_partitions_store, + backfill_datetime, + ) elif partitions_by_assets is not None: if backfill_params.get("forceSynchronousSubmission"): raise DagsterError( @@ -252,16 +268,18 @@ def create_and_launch_partition_backfill( asset_graph=asset_graph, backfill_timestamp=backfill_timestamp, tags=tags, - dynamic_partitions_store=CachingInstanceQueryer( - instance=graphene_info.context.instance, - asset_graph=asset_graph, - loading_context=graphene_info.context, - evaluation_time=datetime_from_timestamp(backfill_timestamp), - ), + dynamic_partitions_store=dynamic_partitions_store, partitions_by_assets=partitions_by_assets, title=backfill_params.get("title"), description=backfill_params.get("description"), ) + assert_valid_asset_partition_backfill( + graphene_info, + backfill, + dynamic_partitions_store, + backfill_datetime, + ) + else: raise DagsterError( "Backfill requested without specifying partition set selector or asset selection" diff --git a/python_modules/dagster-graphql/dagster_graphql/implementation/utils.py b/python_modules/dagster-graphql/dagster_graphql/implementation/utils.py index 0718c497eb7e8..040a228511264 100644 --- a/python_modules/dagster-graphql/dagster_graphql/implementation/utils.py +++ b/python_modules/dagster-graphql/dagster_graphql/implementation/utils.py @@ -2,6 +2,7 @@ import sys from contextlib import contextmanager from contextvars import ContextVar +from datetime import datetime from types import TracebackType from typing import ( TYPE_CHECKING, @@ -25,9 +26,12 @@ import dagster._check as check from dagster._core.definitions.asset_check_spec import AssetCheckKey from dagster._core.definitions.events import AssetKey +from dagster._core.definitions.partition import PartitionsDefinition from dagster._core.definitions.remote_asset_graph import RemoteAssetGraph from dagster._core.definitions.selector import GraphSelector, JobSubsetSelector +from dagster._core.execution.backfill import PartitionBackfill from dagster._core.workspace.context import BaseWorkspaceRequestContext +from dagster._utils.caching_instance_queryer import CachingInstanceQueryer from dagster._utils.error import serializable_error_info_from_exc_info from typing_extensions import ParamSpec, TypeAlias @@ -131,6 +135,60 @@ def assert_permission_for_asset_graph( raise UserFacingGraphQLError(GrapheneUnauthorizedError()) +def assert_valid_job_partition_backfill( + graphene_info: "ResolveInfo", + backfill: PartitionBackfill, + partitions_def: PartitionsDefinition, + dynamic_partitions_store: CachingInstanceQueryer, + backfill_datetime: datetime, +) -> None: + from dagster_graphql.schema.errors import GraphenePartitionKeysNotFoundError + + partition_names = backfill.get_partition_names(graphene_info.context) + + if not partition_names: + return + + invalid_keys = set(partition_names) - set( + partitions_def.get_partition_keys(backfill_datetime, dynamic_partitions_store) + ) + + if invalid_keys: + raise UserFacingGraphQLError(GraphenePartitionKeysNotFoundError(invalid_keys)) + + +def assert_valid_asset_partition_backfill( + graphene_info: "ResolveInfo", + backfill: PartitionBackfill, + dynamic_partitions_store: CachingInstanceQueryer, + backfill_datetime: datetime, +) -> None: + from dagster_graphql.schema.errors import GraphenePartitionKeysNotFoundError + + asset_graph = graphene_info.context.asset_graph + asset_backfill_data = backfill.asset_backfill_data + + if not asset_backfill_data: + return + + partition_subset_by_asset_key = ( + asset_backfill_data.target_subset.partitions_subsets_by_asset_key + ) + + for asset_key, partition_subset in partition_subset_by_asset_key.items(): + partitions_def = asset_graph.get(asset_key).partitions_def + + if not partitions_def: + continue + + invalid_keys = set(partition_subset.get_partition_keys()) - set( + partitions_def.get_partition_keys(backfill_datetime, dynamic_partitions_store) + ) + + if invalid_keys: + raise UserFacingGraphQLError(GraphenePartitionKeysNotFoundError(invalid_keys)) + + def _noop(_) -> None: pass diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py b/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py index bfc2101e583f2..59d9a12420f98 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/backfill.py @@ -39,6 +39,7 @@ GrapheneInvalidOutputError, GrapheneInvalidStepError, GrapheneInvalidSubsetError, + GraphenePartitionKeysNotFoundError, GraphenePartitionSetNotFoundError, GraphenePipelineNotFoundError, GraphenePythonError, @@ -83,6 +84,7 @@ class Meta: types = ( GrapheneLaunchBackfillSuccess, GraphenePartitionSetNotFoundError, + GraphenePartitionKeysNotFoundError, ) + pipeline_execution_error_types name = "LaunchBackfillResult" diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/errors.py b/python_modules/dagster-graphql/dagster_graphql/schema/errors.py index 983119d976487..ddb3ff4b36aeb 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/errors.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/errors.py @@ -1,4 +1,4 @@ -from typing import List, Optional +from typing import List, Optional, Set import dagster._check as check import graphene @@ -451,6 +451,21 @@ def __init__(self, partition_set_name): self.message = f"Partition set {self.partition_set_name} could not be found." +class GraphenePartitionKeysNotFoundError(graphene.ObjectType): + class Meta: + interfaces = (GrapheneError,) + name = "PartitionKeysNotFoundError" + + partition_keys = non_null_list(graphene.String) + + def __init__(self, partition_keys: Set[str]): + super().__init__() + self.partition_keys = check.list_param( + sorted(partition_keys), "partition_keys", of_type=str + ) + self.message = f"Partition keys `{self.partition_keys}` could not be found." + + class GrapheneRepositoryNotFoundError(graphene.ObjectType): class Meta: interfaces = (GrapheneError,) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_asset_backfill.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_asset_backfill.py index 9a55dde61b553..de5a4468a2416 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_asset_backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_asset_backfill.py @@ -576,6 +576,33 @@ def test_launch_asset_backfill(): ) +def test_launch_asset_backfill_with_nonexistent_partition_key(): + repo = get_repo() + all_asset_keys = repo.asset_graph.materializable_asset_keys + + with instance_for_test() as instance: + with define_out_of_process_context(__file__, "get_repo", instance) as context: + # launchPartitionBackfill + launch_backfill_result = execute_dagster_graphql( + context, + LAUNCH_PARTITION_BACKFILL_MUTATION, + variables={ + "backfillParams": { + "partitionNames": ["a", "nonexistent1", "nonexistent2"], + "assetSelection": [key.to_graphql_input() for key in all_asset_keys], + } + }, + ) + assert ( + launch_backfill_result.data["launchPartitionBackfill"]["__typename"] + == "PartitionKeysNotFoundError" + ) + assert ( + "Partition keys `['nonexistent1', 'nonexistent2']` could not be found" + in launch_backfill_result.data["launchPartitionBackfill"]["message"] + ) + + def test_remove_partitions_defs_after_backfill_backcompat(): repo = get_repo() all_asset_keys = repo.asset_graph.materializable_asset_keys diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py index 932ea2e3cd698..3561f3d5edf68 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_partition_backfill.py @@ -1254,6 +1254,31 @@ def test_set_title_and_description_for_backfill_invalid_title(self, graphql_cont in result.data["launchPartitionBackfill"]["message"] ) + def test_asset_job_backfill_with_nonexistent_partition_key(self, graphql_context): + repository_selector = infer_repository_selector(graphql_context) + # launch a backfill for this partition set + launch_result = execute_dagster_graphql( + graphql_context, + LAUNCH_PARTITION_BACKFILL_MUTATION, + variables={ + "backfillParams": { + "selector": { + "repositorySelector": repository_selector, + "partitionSetName": "integers_partition_set", + }, + "partitionNames": ["1", "nonexistent1", "nonexistent2"], + } + }, + ) + assert ( + launch_result.data["launchPartitionBackfill"]["__typename"] + == "PartitionKeysNotFoundError" + ) + assert ( + "Partition keys `['nonexistent1', 'nonexistent2']` could not be found" + in launch_result.data["launchPartitionBackfill"]["message"] + ) + class TestLaunchDaemonBackfillFromFailure(ExecutingGraphQLContextTestMatrix): def test_launch_from_failure(self, graphql_context):