Skip to content

Commit

Permalink
[dagit] Check for asset group job prefix to support multiple groups p…
Browse files Browse the repository at this point in the history
…er repo (#7418)
  • Loading branch information
bengotow committed Apr 14, 2022
1 parent 0bc57e9 commit e0b991b
Show file tree
Hide file tree
Showing 20 changed files with 59 additions and 49 deletions.
18 changes: 11 additions & 7 deletions js_modules/dagit/packages/core/src/app/FallthroughRoot.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ import {Box, ExternalAnchorButton, Colors, NonIdealState, Spinner} from '@dagste
import * as React from 'react';
import {Redirect, Route, Switch, useLocation} from 'react-router-dom';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {WorkspaceContext} from '../workspace/WorkspaceContext';
import {workspacePipelinePath} from '../workspace/workspacePath';

Expand Down Expand Up @@ -51,14 +51,18 @@ const FinalRedirectOrLoadingRoot = () => {

const reposWithAJob = allRepos.filter((r) => r.repository.pipelines.length > 0);

// If we have exactly one job, route to it's overview / graph tab or
// to the asset graph if it's an __ASSET_GROUP job.
// If every loaded repo only contains asset jobs, route to the asset graph
if (
reposWithAJob.every(({repository}) => repository.pipelines.every((p) => isAssetGroup(p.name)))
) {
return <Redirect to="/instance/asset-graph" />;
}

// If we have exactly one repo with one job, route to the job overview
if (reposWithAJob.length === 1 && reposWithAJob[0].repository.pipelines.length === 1) {
const repo = reposWithAJob[0];
const job = repo.repository.pipelines[0];
return job.name === __ASSET_GROUP ? (
<Redirect to="/instance/asset-graph" />
) : (
return (
<Redirect
to={workspacePipelinePath({
repoName: repo.repository.name,
Expand All @@ -70,7 +74,7 @@ const FinalRedirectOrLoadingRoot = () => {
);
}

// If we have more than one job, route to the instance overview
// If we have more than one repo with a job, route to the instance overview
if (reposWithAJob.length > 0) {
return <Redirect to="/instance" />;
}
Expand Down
8 changes: 4 additions & 4 deletions js_modules/dagit/packages/core/src/asset-graph/AssetNode.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import {markdownToPlaintext} from '../ui/markdownToPlaintext';
import {buildRepoAddress} from '../workspace/buildRepoAddress';
import {workspacePath, workspacePipelinePathGuessRepo} from '../workspace/workspacePath';

import {displayNameForAssetKey, LiveDataForNode, __ASSET_GROUP} from './Utils';
import {displayNameForAssetKey, LiveDataForNode, isAssetGroup} from './Utils';
import {AssetNodeFragment} from './types/AssetNodeFragment';

export const AssetNode: React.FC<{
Expand Down Expand Up @@ -78,7 +78,9 @@ export const AssetNode: React.FC<{
{runOrError?.__typename === 'Run' && event ? (
<>
<StatsRow>
{runOrError.pipelineName !== __ASSET_GROUP ? (
{isAssetGroup(runOrError.pipelineName) ? (
<span />
) : (
<Link
data-tooltip={runOrError.pipelineName}
data-tooltip-style={RunLinkTooltipStyle}
Expand All @@ -97,8 +99,6 @@ export const AssetNode: React.FC<{
>
{runOrError.pipelineName}
</Link>
) : (
<span />
)}
<Link
style={{fontFamily: FontFamily.monospace, fontSize: 14}}
Expand Down
6 changes: 5 additions & 1 deletion js_modules/dagit/packages/core/src/asset-graph/Utils.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,11 @@ import {
type AssetNode = AssetGraphQuery_assetNodes;
type AssetKey = AssetGraphQuery_assetNodes_assetKey;

export const __ASSET_GROUP = '__ASSET_GROUP';
export const __ASSET_GROUP_PREFIX = '__ASSET_GROUP';

export function isAssetGroup(jobName: string) {
return jobName.startsWith(__ASSET_GROUP_PREFIX);
}

// IMPORTANT: We use this, rather than AssetNode.id throughout this file because
// the GraphQL interface exposes dependencyKeys, not dependencyIds. We also need
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {Timestamp} from '../app/time/Timestamp';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {MetadataEntry} from '../metadata/MetadataEntry';
import {PipelineReference} from '../pipelines/PipelineReference';
import {RunStatusWithStats} from '../runs/RunStatusDots';
Expand Down Expand Up @@ -231,7 +231,7 @@ const EventGroupRow: React.FC<{
</Group>
</td>
<td>
{run.pipelineName !== __ASSET_GROUP && (
{!isAssetGroup(run.pipelineName) && (
<Box margin={{bottom: 4}}>
<Box padding={{left: 8}}>
<PipelineReference
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@ import {gql} from '@apollo/client';
import {Box, MetadataTable} from '@dagster-io/ui';
import * as React from 'react';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {DAGSTER_TYPE_FRAGMENT} from '../dagstertype/DagsterType';
import {DagsterTypeFragment} from '../dagstertype/types/DagsterTypeFragment';
import {MetadataEntry, METADATA_ENTRY_FRAGMENT} from '../metadata/MetadataEntry';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ import {
isSourceAsset,
LiveData,
tokenForAssetKey,
__ASSET_GROUP,
isAssetGroup,
} from '../asset-graph/Utils';
import {DagsterTypeSummary} from '../dagstertype/DagsterType';
import {Description} from '../pipelines/Description';
Expand Down Expand Up @@ -172,7 +172,7 @@ const DefinitionLocation: React.FC<{
}> = ({assetNode, repoAddress}) => (
<Box flex={{alignItems: 'baseline', gap: 16, wrap: 'wrap'}} style={{lineHeight: 0}}>
{assetNode.jobNames
.filter((jobNames) => jobNames !== __ASSET_GROUP)
.filter((jobName) => !isAssetGroup(jobName))
.map((jobName) => (
<Mono key={jobName}>
<PipelineReference
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ import {Warning} from '@dagster-io/ui';
import React from 'react';
import {Link} from 'react-router-dom';

import {LiveDataForNode, __ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup, LiveDataForNode} from '../asset-graph/Utils';
import {titleForRun} from '../runs/RunUtils';
import {runsPathWithFilters} from '../runs/RunsFilterInput';

Expand All @@ -14,7 +14,7 @@ export const FailedRunsSinceMaterializationBanner: React.FC<{liveData?: LiveData
if (runsSinceMaterialization) {
const {jobNames, count} = runsSinceMaterialization;
const jobNamesCleaned = jobNames.map((j) =>
j === __ASSET_GROUP ? 'Asset materialization runs' : j,
isAssetGroup(j) ? 'Asset materialization runs' : j,
);
const jobNamesSummary =
jobNamesCleaned.length > 1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {Timestamp} from '../app/time/Timestamp';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {MetadataEntry, METADATA_ENTRY_FRAGMENT} from '../metadata/MetadataEntry';
import {PipelineReference} from '../pipelines/PipelineReference';
import {titleForRun} from '../runs/RunUtils';
Expand Down Expand Up @@ -57,7 +57,7 @@ export const LatestMaterializationMetadata: React.FC<{
<Mono>{titleForRun({runId: latestEvent.runId})}</Mono>
</Link>
</Box>
{latestRun.pipelineName !== __ASSET_GROUP && (
{!isAssetGroup(latestRun.pipelineName) && (
<>
<Box padding={{left: 8, top: 4}}>
<PipelineReference
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,12 @@ export const LaunchAssetExecutionButton: React.FC<{
disabledReason || 'Assets must be in the same repository to be materialized together.';
}

const partitionDefinition = assets.find((a) => !!a.partitionDefinition)?.partitionDefinition;
if (assets.some((a) => a.partitionDefinition && a.partitionDefinition !== partitionDefinition)) {
disabledReason =
disabledReason || 'Assets must share a partition definition to be materialized together.';
}

const everyAssetHasJob = (jobName: string) => assets.every((a) => a.jobNames.includes(jobName));
const jobsInCommon = assets[0] ? assets[0].jobNames.filter(everyAssetHasJob) : [];
const jobName = jobsInCommon.find((name) => name === preferredJobName) || jobsInCommon[0];
Expand All @@ -52,12 +58,6 @@ export const LaunchAssetExecutionButton: React.FC<{
disabledReason || 'Assets must be in the same job to be materialized together.';
}

const partitionDefinition = assets[0]?.partitionDefinition;
if (assets.some((a) => a.partitionDefinition !== partitionDefinition)) {
disabledReason =
disabledReason || 'Assets must share a partition definition to be materialized together.';
}

title = title || 'Refresh';
if (partitionDefinition) {
// Add ellipsis to the button title since it will open a "Choose partitions" modal
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import * as React from 'react';

import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorInfo';
import {FIFTEEN_SECONDS, useQueryRefreshAtInterval} from '../app/QueryRefresh';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {ScheduleOrSensorTag} from '../nav/ScheduleOrSensorTag';
import {LegacyPipelineTag} from '../pipelines/LegacyPipelineTag';
import {PipelineReference} from '../pipelines/PipelineReference';
Expand Down Expand Up @@ -207,7 +207,7 @@ export const InstanceOverviewPage = () => {
r.repositoryLocation.name === repoAddress.location,
) &&
job.name.toLocaleLowerCase().includes(searchToLower) &&
job.name !== __ASSET_GROUP;
!isAssetGroup(job.name);

const {failed, inProgress, queued, succeeded, neverRan} = bucketed;
return {
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/nav/FlatContentList.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ import * as React from 'react';
import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {LegacyPipelineTag} from '../pipelines/LegacyPipelineTag';
import {InstigationStatus} from '../types/globalTypes';
import {
Expand Down Expand Up @@ -55,7 +55,7 @@ export const FlatContentList: React.FC<Props> = (props) => {

const {schedules, sensors} = repository;
for (const pipeline of repository.pipelines) {
if (pipeline.name === __ASSET_GROUP) {
if (isAssetGroup(pipeline.name)) {
continue;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ import {Mono} from '@dagster-io/ui';
import React from 'react';
import {Link, useHistory} from 'react-router-dom';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {__ASSET_GROUP_PREFIX} from '../asset-graph/Utils';

export interface ExplorerPath {
pipelineName: string;
Expand Down Expand Up @@ -48,13 +48,16 @@ export function explorerPathFromString(path: string): ExplorerPath {
export function instanceAssetsExplorerPathFromString(path: string): ExplorerPath {
// This is a bit of a hack, but our explorer path needs a job name and we'd like
// to continue sharing the parsing/stringifying logic from the job graph UI
return explorerPathFromString(__ASSET_GROUP + path || '/');
return explorerPathFromString(__ASSET_GROUP_PREFIX + path || '/');
}

export function instanceAssetsExplorerPathToURL(path: Omit<ExplorerPath, 'pipelineName'>) {
return (
'/instance/asset-graph' +
explorerPathToString({...path, pipelineName: __ASSET_GROUP}).replace(__ASSET_GROUP, '')
explorerPathToString({...path, pipelineName: __ASSET_GROUP_PREFIX}).replace(
__ASSET_GROUP_PREFIX,
'',
)
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {breakOnUnderscores} from '../app/Util';
import {displayNameForAssetKey, __ASSET_GROUP} from '../asset-graph/Utils';
import {displayNameForAssetKey, isAssetGroup} from '../asset-graph/Utils';
import {OpTypeSignature, OP_TYPE_SIGNATURE_FRAGMENT} from '../ops/OpTypeSignature';
import {pluginForMetadata} from '../plugins';
import {ConfigTypeSchema, CONFIG_TYPE_SCHEMA_FRAGMENT} from '../typeexplorer/ConfigTypeSchema';
Expand Down Expand Up @@ -258,7 +258,7 @@ const InvocationList: React.FC<{
onClickInvocation: (arg: SidebarOpInvocationInfo) => void;
}> = ({invocations, onClickInvocation}) => {
const [showAll, setShowAll] = React.useState<boolean>(false);
const visible = invocations.filter((i) => i.pipelineName !== __ASSET_GROUP);
const visible = invocations.filter((i) => !isAssetGroup(i.pipelineName || ''));
const clipped = showAll ? visible : visible.slice(0, DEFAULT_INVOCATIONS_SHOWN);

return (
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/runs/Run.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ import styled from 'styled-components/macro';
import {showCustomAlert} from '../app/CustomAlertProvider';
import {filterByQuery} from '../app/GraphQueryImpl';
import {PythonErrorInfo} from '../app/PythonErrorInfo';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {GanttChart, GanttChartLoadingState, GanttChartMode, QueuedState} from '../gantt/GanttChart';
import {toGraphQueryItems} from '../gantt/toGraphQueryItems';
import {useDocumentTitle} from '../hooks/useDocumentTitle';
Expand Down Expand Up @@ -57,7 +57,7 @@ export const Run: React.FC<RunProps> = (props) => {
useFavicon(run ? runStatusFavicon(run.status) : '/favicon.svg');
useDocumentTitle(
run
? `${run.pipelineName !== __ASSET_GROUP ? run.pipelineName : ''} ${runId.slice(0, 8)} [${
? `${!isAssetGroup(run.pipelineName) ? run.pipelineName : ''} ${runId.slice(0, 8)} [${
run.status
}]`
: `Run: ${runId}`,
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/runs/RunRoot.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import * as React from 'react';
import {useParams} from 'react-router-dom';

import {formatElapsedTime} from '../app/Util';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {PipelineReference} from '../pipelines/PipelineReference';
import {TimestampDisplay} from '../schedules/TimestampDisplay';
import {isThisThingAJob} from '../workspace/WorkspaceContext';
Expand Down Expand Up @@ -68,7 +68,7 @@ export const RunRoot = () => {
run ? (
<>
<RunStatusTag status={run.status} />
{run.pipelineName !== __ASSET_GROUP ? (
{!isAssetGroup(run.pipelineName) ? (
<Tag icon="run">
Run of{' '}
<PipelineReference
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/runs/RunTable.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {usePermissions} from '../app/Permissions';
import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {useSelectionReducer} from '../hooks/useSelectionReducer';
import {PipelineSnapshotLink} from '../pipelines/PipelinePathUtils';
import {PipelineReference} from '../pipelines/PipelineReference';
Expand Down Expand Up @@ -223,7 +223,7 @@ const RunRow: React.FC<{
</td>
<td>
<Box flex={{direction: 'column', gap: 5}}>
{run.pipelineName !== __ASSET_GROUP ? (
{!isAssetGroup(run.pipelineName) ? (
<Box flex={{direction: 'row', gap: 8, alignItems: 'center'}}>
<PipelineReference
isJob={isJob}
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/search/useRepoSearch.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ import Fuse from 'fuse.js';
import * as React from 'react';

import {PYTHON_ERROR_FRAGMENT} from '../app/PythonErrorInfo';
import {displayNameForAssetKey, __ASSET_GROUP} from '../asset-graph/Utils';
import {displayNameForAssetKey, isAssetGroup} from '../asset-graph/Utils';
import {buildRepoPath} from '../workspace/buildRepoAddress';
import {workspacePath} from '../workspace/workspacePath';

Expand Down Expand Up @@ -65,7 +65,7 @@ const bootstrapDataToSearchResults = (data?: SearchBootstrapQuery) => {
},
];
}, [] as SearchResult[])
.filter((item) => item.label !== __ASSET_GROUP);
.filter((item) => !isAssetGroup(item.label));

const allSchedules: SearchResult[] = schedules.map((schedule) => ({
key: `${repoPath}-${schedule.name}`,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import * as React from 'react';
import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {__ASSET_GROUP, displayNameForAssetKey} from '../asset-graph/Utils';
import {displayNameForAssetKey} from '../asset-graph/Utils';
import {RepositoryLink} from '../nav/RepositoryLink';

import {repoAddressAsString} from './repoAddressAsString';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import * as React from 'react';
import {Link} from 'react-router-dom';
import styled from 'styled-components/macro';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';

import {repoAddressAsString} from './repoAddressAsString';
import {repoAddressToSelector} from './repoAddressToSelector';
Expand Down Expand Up @@ -77,7 +77,7 @@ export const RepositoryGraphsList: React.FC<Props> = (props) => {
return null;
}
const jobGraphNames = new Set<string>(
repo.pipelines.filter((p) => p.isJob && p.name !== __ASSET_GROUP).map((p) => p.graphName),
repo.pipelines.filter((p) => p.isJob && !isAssetGroup(p.name)).map((p) => p.graphName),
);
const items: Item[] = Array.from(jobGraphNames).map((graphName) => ({
name: graphName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ import {gql, useQuery} from '@apollo/client';
import {Box, NonIdealState} from '@dagster-io/ui';
import * as React from 'react';

import {__ASSET_GROUP} from '../asset-graph/Utils';
import {isAssetGroup} from '../asset-graph/Utils';
import {PipelineTable, PIPELINE_TABLE_FRAGMENT} from '../pipelines/PipelineTable';

import {repoAddressAsString} from './repoAddressAsString';
Expand Down Expand Up @@ -52,7 +52,7 @@ export const RepositoryPipelinesList: React.FC<Props> = (props) => {
return null;
}
return repo.pipelines
.filter((pipelineOrJob) => pipelineOrJob.name !== __ASSET_GROUP)
.filter((pipelineOrJob) => !isAssetGroup(pipelineOrJob.name))
.map((pipelineOrJob) => ({
pipelineOrJob,
repoAddress,
Expand Down

0 comments on commit e0b991b

Please sign in to comment.