Skip to content

Commit

Permalink
Handle InvalidSubsetError in launch run results (#8014)
Browse files Browse the repository at this point in the history
Summary:
A user reported that a launch run mutation that triggered an InvalidSubsetError was raised as an unhandled graphql exception. Add that explicitly to the set of errors that are handled in the graphql query and client-side.

Not sure how I feel about this java-y "catch all exceptions that might possibly be thrown from the server" pattern though.

There's also a visual display issue with this showCustomAlert path when there's only a message but no stack trace - i'll post a screenshot. To repro swap in this implementation on the server:

```
@capture_error
def launch_pipeline_execution(graphene_info, execution_params):
    from ...schema.errors import GrapheneInvalidSubsetError
    from ...schema.pipelines.pipeline import GraphenePipeline
    from ..utils import UserFacingGraphQLError, capture_error

    raise UserFacingGraphQLError(
        GrapheneInvalidSubsetError(
            message="sup",
            pipeline=GraphenePipeline(
                graphene_info.context.get_full_external_pipeline(execution_params.selector)
            ),
        ),
    )
    return _launch_pipeline_execution(graphene_info, execution_params)
```

Test Plan: Simulate this error being thrown from the server using the error above, launch run - no more uncaught graphql exception.
  • Loading branch information
gibsondan committed May 24, 2022
1 parent aebfe43 commit e385dec
Show file tree
Hide file tree
Showing 17 changed files with 87 additions and 56 deletions.

Large diffs are not rendered by default.

15 changes: 9 additions & 6 deletions js_modules/dagit/packages/core/src/graphql/schema.graphql

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 4 additions & 0 deletions js_modules/dagit/packages/core/src/runs/ReexecutionDialog.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import {
LaunchPipelineReexecution_launchPipelineReexecution_PipelineNotFoundError,
LaunchPipelineReexecution_launchPipelineReexecution_PythonError,
LaunchPipelineReexecution_launchPipelineReexecution_RunConfigValidationInvalid,
LaunchPipelineReexecution_launchPipelineReexecution_InvalidSubsetError,
} from './types/LaunchPipelineReexecution';

export interface Props {
Expand All @@ -29,6 +30,7 @@ type Error =
| LaunchPipelineReexecution_launchPipelineReexecution_InvalidStepError
| LaunchPipelineReexecution_launchPipelineReexecution_PipelineNotFoundError
| LaunchPipelineReexecution_launchPipelineReexecution_RunConfigValidationInvalid
| LaunchPipelineReexecution_launchPipelineReexecution_InvalidSubsetError
| LaunchPipelineReexecution_launchPipelineReexecution_PythonError
| undefined;

Expand Down Expand Up @@ -57,6 +59,8 @@ const errorText = (error: Error) => {
return 'Run conflict';
case 'UnauthorizedError':
return 'Re-execution not authorized';
case 'InvalidSubsetError':
return 'Invalid op subset: ' + error.message;
default:
return 'Unknown error';
}
Expand Down
8 changes: 8 additions & 0 deletions js_modules/dagit/packages/core/src/runs/RunUtils.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,8 @@ export function handleLaunchResult(
});
}
document.dispatchEvent(new CustomEvent('run-launched'));
} else if (obj.__typename === 'InvalidSubsetError') {
showCustomAlert({body: obj.message});
} else if (obj.__typename === 'PythonError') {
showCustomAlert({
title: 'Error',
Expand Down Expand Up @@ -213,6 +215,9 @@ export const LAUNCH_PIPELINE_EXECUTION_MUTATION = gql`
... on PipelineNotFoundError {
message
}
... on InvalidSubsetError {
message
}
... on RunConfigValidationInvalid {
errors {
message
Expand Down Expand Up @@ -291,6 +296,9 @@ export const LAUNCH_PIPELINE_REEXECUTION_MUTATION = gql`
... on PipelineNotFoundError {
message
}
... on InvalidSubsetError {
message
}
... on RunConfigValidationInvalid {
errors {
message
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ def get_external_pipeline_or_raise(graphene_info, selector):


def get_subset_external_pipeline(context, selector):
from ..schema.errors import GrapheneInvalidSubsetError
from ..schema.pipelines.pipeline import GraphenePipeline
from ..schema.pipelines.pipeline_errors import GrapheneInvalidSubsetError

check.inst_param(selector, "selector", PipelineSelector)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
from .errors import (
GrapheneInvalidOutputError,
GrapheneInvalidStepError,
GrapheneInvalidSubsetError,
GraphenePartitionSetNotFoundError,
GraphenePipelineNotFoundError,
GraphenePythonError,
Expand All @@ -27,6 +28,7 @@
GrapheneRunConflict,
GrapheneUnauthorizedError,
GraphenePythonError,
GrapheneInvalidSubsetError,
) + create_execution_params_error_types


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,11 @@
GrapheneEnumConfigType,
GrapheneRegularConfigType,
)
from .errors import GraphenePipelineNotFoundError, GraphenePythonError
from .pipelines.pipeline_errors import GrapheneConfigTypeNotFoundError
from .errors import (
GrapheneConfigTypeNotFoundError,
GraphenePipelineNotFoundError,
GraphenePythonError,
)


class GrapheneConfigTypeOrError(graphene.Union):
Expand Down
28 changes: 28 additions & 0 deletions python_modules/dagster-graphql/dagster_graphql/schema/errors.py
Original file line number Diff line number Diff line change
Expand Up @@ -249,6 +249,32 @@ class Meta:
name = "RunConflict"


class GrapheneInvalidSubsetError(graphene.ObjectType):
class Meta:
interfaces = (GrapheneError,)
name = "InvalidSubsetError"

pipeline = graphene.Field(
graphene.NonNull("dagster_graphql.schema.pipelines.pipeline.GraphenePipeline")
)

def __init__(self, message, pipeline):
super().__init__()
self.message = check.str_param(message, "message")
self.pipeline = pipeline


class GrapheneConfigTypeNotFoundError(graphene.ObjectType):
class Meta:
interfaces = (GrapheneError,)
name = "ConfigTypeNotFoundError"

pipeline = graphene.Field(
graphene.NonNull("dagster_graphql.schema.pipelines.pipeline.GraphenePipeline")
)
config_type_name = graphene.NonNull(graphene.String)


create_execution_params_error_types = (
GraphenePresetNotFoundError,
GrapheneConflictingExecutionParamsError,
Expand Down Expand Up @@ -344,11 +370,13 @@ def __init__(self, message=None):
types = [
GrapheneAssetNotFoundError,
GrapheneConflictingExecutionParamsError,
GrapheneConfigTypeNotFoundError,
GrapheneDagsterTypeNotFoundError,
GrapheneError,
GrapheneInvalidOutputError,
GrapheneInvalidPipelineRunsFilterError,
GrapheneInvalidStepError,
GrapheneInvalidSubsetError,
GrapheneModeNotFoundError,
GrapheneNoModeProvidedError,
GraphenePartitionSetNotFoundError,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ def types():
GrapheneRun,
GrapheneRunOrError,
)
from .pipeline_errors import GrapheneConfigTypeNotFoundError, GrapheneInvalidSubsetError
from .pipeline_ref import GraphenePipelineReference, GrapheneUnknownPipeline
from .pipeline_run_stats import (
GraphenePipelineRunStatsSnapshot,
Expand All @@ -48,7 +47,6 @@ def types():

return [
GrapheneAsset,
GrapheneConfigTypeNotFoundError,
GrapheneEvaluationErrorReason,
GrapheneEvaluationStack,
GrapheneEvaluationStackEntry,
Expand All @@ -58,7 +56,6 @@ def types():
GrapheneEvaluationStackMapValueEntry,
GrapheneFieldNotDefinedConfigError,
GrapheneFieldsNotDefinedConfigError,
GrapheneInvalidSubsetError,
GrapheneIPipelineSnapshot,
GrapheneLogger,
GrapheneMissingFieldConfigError,
Expand Down
Original file line number Diff line number Diff line change
@@ -1,9 +1,8 @@
# pylint: disable=missing-graphene-docstring
import graphene

from ..errors import GraphenePipelineNotFoundError, GraphenePythonError
from ..errors import GrapheneInvalidSubsetError, GraphenePipelineNotFoundError, GraphenePythonError
from .config import GraphenePipelineConfigValidationValid, GrapheneRunConfigValidationInvalid
from .pipeline_errors import GrapheneInvalidSubsetError


class GraphenePipelineConfigValidationResult(graphene.Union):
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -1,10 +1,9 @@
# pylint: disable=missing-graphene-docstring
import graphene

from ..errors import GraphenePipelineNotFoundError, GraphenePythonError
from ..errors import GrapheneInvalidSubsetError, GraphenePipelineNotFoundError, GraphenePythonError
from ..execution import GrapheneExecutionPlan
from ..pipelines.config import GrapheneRunConfigValidationInvalid
from ..pipelines.pipeline_errors import GrapheneInvalidSubsetError


class GrapheneExecutionPlanOrError(graphene.Union):
Expand Down
Original file line number Diff line number Diff line change
@@ -1,9 +1,13 @@
# pylint: disable=missing-graphene-docstring
import graphene

from ..errors import GrapheneGraphNotFoundError, GraphenePipelineNotFoundError, GraphenePythonError
from ..errors import (
GrapheneGraphNotFoundError,
GrapheneInvalidSubsetError,
GraphenePipelineNotFoundError,
GraphenePythonError,
)
from ..pipelines.pipeline import GrapheneGraph, GraphenePipeline
from ..pipelines.pipeline_errors import GrapheneInvalidSubsetError


class GraphenePipelineOrError(graphene.Union):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,12 @@

from ..implementation.run_config_schema import resolve_is_run_config_valid
from .config_types import GrapheneConfigType, to_config_type
from .errors import GrapheneModeNotFoundError, GraphenePipelineNotFoundError, GraphenePythonError
from .errors import (
GrapheneInvalidSubsetError,
GrapheneModeNotFoundError,
GraphenePipelineNotFoundError,
GraphenePythonError,
)
from .pipelines.config_result import GraphenePipelineConfigValidationResult
from .runs import GrapheneRunConfigData, parse_run_config_input
from .util import non_null_list
Expand Down Expand Up @@ -82,8 +87,6 @@ def resolve_isRunConfigValid(self, graphene_info, **kwargs):

class GrapheneRunConfigSchemaOrError(graphene.Union):
class Meta:
from .pipelines.pipeline_errors import GrapheneInvalidSubsetError

types = (
GrapheneRunConfigSchema,
GraphenePipelineNotFoundError,
Expand Down

0 comments on commit e385dec

Please sign in to comment.