Skip to content

Commit

Permalink
Improve hot-reloading, display Python errors in Dagit UI (#242)
Browse files Browse the repository at this point in the history
* Capture, expose most recent python error to GraphQL

* Simplify implementation of dynamic loading, handle case where initial load fails

The problem with the current implementation of DynamicObject is that the tuple contains both the `module` reference and also the evaluated result. If you can’t evaluate the function, you can’t obtain a DynamicObject, which means you have no module handle to use to call `importlib.reload`.

To fix this I’ve split the loading into a two phase process—we create a `DynamicObject` with just the module and function name, and then evaluate the module separately. This allows 1) the evaluation to be done in the try catch and 2) means the initial load can fail and still have a DynamicObject with a module reference. I also made `eval` idempotent so it can also replace `reload_dynamic_object`.

* Display errors in the React UI

* Fix the tests

* Fix tests, allow RepositoryContainer(repository=obj) for inline repos

* Put the new error field in the mock graphql response to fix JS test

* DynamicLoader eval => load

* Very slightly move code coercing pipeline to repo

* Expose pipelinesOrErrors, pipelineOrError at top level of graphql schema

* Client-side updates to use pipelinesOrErrors

* Use the reloader module to build a dependency graph, reload everything in order

* Fix the web app mock query data

* Do a bit of linting

* Handle non-pipeline, non-repository case with fall through exception

* Included reloader in the wrong python setup.py

* On Python 2, disable module reloading by stubbing `reloader`

* Only install reloader on python3, setup script isn’t even compatible with 2.x

* Reload dagster itself (though it is not in the watched path)

* check.inst_param => check.inst

* Just throw rather than nicely returning if no repository is present

* Revert "Very slightly move code coercing pipeline to repo"

This reverts commit 24548f6.

# Conflicts:
#	python_modules/dagster/dagster/cli/dynamic_loader.py

* Explicitly check for two supported types

* Fix tests

* Add comment
  • Loading branch information
bengotow authored and schrockn committed Oct 31, 2018
1 parent 34160fe commit 55a2944
Show file tree
Hide file tree
Showing 13 changed files with 393 additions and 207 deletions.
33 changes: 24 additions & 9 deletions python_modules/dagit/dagit/app.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
import os
import sys
try:
from graphql.execution.executors.asyncio import AsyncioExecutor as Executor
except ImportError:
Expand All @@ -10,8 +11,8 @@
from dagster import check

from dagster.cli.dynamic_loader import (
load_repository_object_from_target_info,
DynamicObject,
reload_pipeline_or_repo,
)

from .schema import create_schema
Expand All @@ -24,20 +25,34 @@ class RepositoryContainer(object):
object allows the RepositoryInfo to be written in an immutable fashion.
'''

def __init__(self, repo_dynamic_obj):
self.repo_dynamic_obj = check.inst_param(
repo_dynamic_obj,
'repo_dynamic_obj',
DynamicObject,
)
def __init__(self, repository_target_info=None, repository=None):
if repository_target_info != None:
self.repo_dynamic_obj = check.inst(
load_repository_object_from_target_info(repository_target_info),
DynamicObject,
)
self.repo = None
self.repo_error = None
self.reload()
elif repository != None:
self.repo = repository

def reload(self):
self.repo_dynamic_obj = reload_pipeline_or_repo(self.repo_dynamic_obj)
if not self.repo_dynamic_obj:
return
try:
self.repo = self.repo_dynamic_obj.load()
self.repo_error = None
except:
self.repo_error = sys.exc_info()

@property
def repository(self):
return self.repo_dynamic_obj.object
return self.repo

@property
def error(self):
return self.repo_error

class DagsterGraphQLView(GraphQLView):
def __init__(self, repository_container, **kwargs):
Expand Down
4 changes: 2 additions & 2 deletions python_modules/dagit/dagit/cli.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ def __init__(self, repository_container):

def on_any_event(self, event):
if event.src_path.endswith('.py'):
print('Reloading repository...')
self.repository_container.reload()


Expand Down Expand Up @@ -60,10 +61,9 @@ def on_any_event(self, event):
)
def ui(host, port, watch, **kwargs):
repository_target_info = load_target_info_from_cli_args(kwargs)
dynamic_obj_repo = load_repository_object_from_target_info(repository_target_info)

sys.path.append(os.getcwd())
repository_container = RepositoryContainer(dynamic_obj_repo)
repository_container = RepositoryContainer(repository_target_info)
if watch:
observer = Observer()
handler = ReloaderHandler(repository_container)
Expand Down
51 changes: 50 additions & 1 deletion python_modules/dagit/dagit/schema.py
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
import graphene

import sys
import traceback
import dagster
import dagster.core.definitions

Expand All @@ -10,7 +11,11 @@

class Query(graphene.ObjectType):
pipeline = graphene.Field(lambda: Pipeline, name=graphene.NonNull(graphene.String))
pipelineOrError = graphene.Field(lambda: PipelineOrError, name=graphene.NonNull(graphene.String))

pipelines = graphene.NonNull(graphene.List(lambda: graphene.NonNull(Pipeline)))
pipelinesOrErrors = graphene.NonNull(graphene.List(lambda: graphene.NonNull(PipelineOrError)))

type = graphene.Field(
lambda: Type,
pipelineName=graphene.NonNull(graphene.String),
Expand All @@ -21,18 +26,42 @@ class Query(graphene.ObjectType):
pipelineName=graphene.NonNull(graphene.String),
)

def result_or_error(self, fn, info, *argv):
error = info.context['repository_container'].error
if error != None:
return PythonError(*error)
try:
return fn(self, info, *argv)
except:
return PythonError(*sys.exc_info())

def results_or_errors(self, fn, info, *argv):
error = info.context['repository_container'].error
if error != None:
return [PythonError(*error)]
try:
return fn(self, info, *argv)
except:
return [PythonError(*sys.exc_info())]

def resolve_pipeline(self, info, name):
check.str_param(name, 'name')
repository = info.context['repository_container'].repository
return Pipeline(repository.get_pipeline(name))

def resolve_pipelineOrError(self, info, name):
return Query.result_or_error(self, Query.resolve_pipeline, info, name)

def resolve_pipelines(self, info):
repository = info.context['repository_container'].repository
pipelines = []
for pipeline_def in repository.get_all_pipelines():
pipelines.append(Pipeline(pipeline_def))
return pipelines

def resolve_pipelinesOrErrors(self, info):
return Query.results_or_errors(self, Query.resolve_pipelines, info)

def resolve_type(self, info, pipelineName, typeName):
check.str_param(pipelineName, 'pipelineName')
check.str_param(typeName, 'typeName')
Expand Down Expand Up @@ -80,6 +109,26 @@ def resolve_environment_type(self, _info):
return Type.from_dagster_type(self._pipeline.environment_type)



class Error(graphene.Interface):
message = graphene.String(required=True)
stack = graphene.NonNull(graphene.List(graphene.NonNull(graphene.String)))


class PythonError(graphene.ObjectType):
class Meta:
interfaces = (Error, )

def __init__(self, exc_type, exc_value, exc_tb):
self.message=traceback.format_exception_only(exc_type, exc_value)[0]
self.stack=traceback.format_tb(tb=exc_tb)


class PipelineOrError(graphene.Union):
class Meta:
types = (Pipeline, PythonError)


class PipelineContext(graphene.ObjectType):
name = graphene.NonNull(graphene.String)
description = graphene.String()
Expand Down
118 changes: 71 additions & 47 deletions python_modules/dagit/dagit/webapp/src/App.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -6,64 +6,82 @@ import { BrowserRouter } from "react-router-dom";
import Loading from "./Loading";
import PipelineExplorer from "./PipelineExplorer";
import PipelineJumpBar from "./PipelineJumpBar";
import { AppQuery } from "./types/AppQuery";
import PythonErrorInfo from "./PythonErrorInfo";
import {
AppQuery,
AppQuery_pipelinesOrErrors,
AppQuery_pipelinesOrErrors_Pipeline
} from "./types/AppQuery";

import { Alignment, Navbar, NonIdealState } from "@blueprintjs/core";
import navBarImage from "./images/nav-logo.png";

function isPipeline(
item: AppQuery_pipelinesOrErrors
): item is AppQuery_pipelinesOrErrors_Pipeline {
return (item as AppQuery_pipelinesOrErrors_Pipeline).solids !== undefined;
}

export default class App extends React.Component {
public render() {
return (
<Query query={APP_QUERY}>
{(queryResult: QueryResult<AppQuery, any>) => (
<Loading queryResult={queryResult}>
{data => (
<BrowserRouter>
<Route path="/:pipeline/:solid?">
{({ match, history }) => {
const selectedPipeline = data.pipelines.find(
p => (match ? p.name === match.params.pipeline : false)
);
const selectedSolid =
selectedPipeline &&
selectedPipeline.solids.find(
s => (match ? s.name === match.params.solid : false)
);
{data => {
const first = data.pipelinesOrErrors[0];
const pipelines = data.pipelinesOrErrors.filter(isPipeline);

return (
<>
<Navbar>
<Navbar.Group align={Alignment.LEFT}>
<Navbar.Heading>
<img src={navBarImage} style={{ height: 34 }} />
</Navbar.Heading>
<Navbar.Divider />
<PipelineJumpBar
selectedPipeline={selectedPipeline}
selectedSolid={selectedSolid}
pipelines={data.pipelines}
return (
<BrowserRouter>
<Route path="/:pipeline/:solid?">
{({ match, history }) => {
const selectedPipeline =
match &&
pipelines.find(p => p.name === match.params.pipeline);
const selectedSolid =
selectedPipeline &&
selectedPipeline.solids.find(
s => s.name === match.params.solid
);

return (
<>
<Navbar>
<Navbar.Group align={Alignment.LEFT}>
<Navbar.Heading>
<img src={navBarImage} style={{ height: 34 }} />
</Navbar.Heading>
<Navbar.Divider />
<PipelineJumpBar
selectedPipeline={selectedPipeline}
selectedSolid={selectedSolid}
pipelines={pipelines}
history={history}
/>
</Navbar.Group>
</Navbar>
{!isPipeline(first) ? (
<PythonErrorInfo error={first} />
) : selectedPipeline ? (
<PipelineExplorer
pipeline={selectedPipeline}
solid={selectedSolid}
history={history}
/>
</Navbar.Group>
</Navbar>
{selectedPipeline ? (
<PipelineExplorer
pipeline={selectedPipeline}
solid={selectedSolid}
history={history}
/>
) : (
<NonIdealState
title="No pipeline selected"
description="Select a pipeline in the sidebar on the left"
/>
)}
</>
);
}}
</Route>
</BrowserRouter>
)}
) : (
<NonIdealState
title="No pipeline selected"
description="Select a pipeline in the sidebar on the left"
/>
)}
</>
);
}}
</Route>
</BrowserRouter>
);
}}
</Loading>
)}
</Query>
Expand All @@ -73,8 +91,14 @@ export default class App extends React.Component {

export const APP_QUERY = gql`
query AppQuery {
pipelines {
...PipelineFragment
pipelinesOrErrors {
... on Error {
message
stack
}
... on Pipeline {
...PipelineFragment
}
}
}
Expand Down
64 changes: 64 additions & 0 deletions python_modules/dagit/dagit/webapp/src/PythonErrorInfo.tsx
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
import * as React from "react";
import styled from "styled-components";
import { Button } from "@blueprintjs/core";

interface IPythonErrorInfoProps {
error: {
message: string;
stack: string[];
};
}

export default class PythonErrorInfo extends React.Component<
IPythonErrorInfoProps,
{}
> {
render() {
const { message, stack } = this.props.error;

return (
<ErrorWrapper>
<ErrorHeader>{message}</ErrorHeader>
<Trace>{stack.join("")}</Trace>
<Button icon="refresh" onClick={() => window.location.reload()}>
Reload
</Button>
</ErrorWrapper>
);
}
}

const ErrorHeader = styled.h3`
color: #b05c47;
font-weight: 400;
margin: 0.5em 0 0.25em;
`;

const BoldMessage = styled.div`
font-weight: 600;
font-family: Consolas, Menlo, monospace;
font-size: 0.95em;
margin-bottom: 0.7em;
`;

const Trace = styled.div`
color: rgb(41, 50, 56);
font-family: Consolas, Menlo, monospace;
font-size: 0.85em;
white-space: pre;
overflow-x: scroll;
padding-bottom: 1em;
`;

const ErrorWrapper = styled.div`
position: absolute;
left: 50%;
top: 100px;
transform: translate(-50%, 0);
margin-bottom: 100px;
background-color: rgba(206, 17, 38, 0.05);
border: 1px solid #d17257;
border-radius: 3px;
max-width: 90vw;
padding: 1em 2em;
`;
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ const mocks = [
},
result: {
data: {
pipelines: [
pipelinesOrErrors: [
{
name: "pandas_hello_world",
description: null,
Expand Down
2 changes: 1 addition & 1 deletion python_modules/dagit/dagit/webapp/src/schema.json

Large diffs are not rendered by default.

Loading

0 comments on commit 55a2944

Please sign in to comment.