Skip to content

Commit

Permalink
Renamed the prefix for auto created jobs from '__ASSET_GROUP' to '__A…
Browse files Browse the repository at this point in the history
…SSET_JOB'. (#8235)

The old name is confusing because we now have asset groups and these jobs are not related to asset groups.
  • Loading branch information
shalabhc committed Jun 7, 2022
1 parent 6498f34 commit 20f5889
Show file tree
Hide file tree
Showing 7 changed files with 19 additions and 18 deletions.
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/asset-graph/Utils.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,10 @@ import {
type AssetNode = AssetGraphQuery_assetNodes;
type AssetKey = AssetGraphQuery_assetNodes_assetKey;

export const __ASSET_GROUP_PREFIX = '__ASSET_GROUP';
export const __ASSET_JOB_PREFIX = '__ASSET_JOB';

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

// IMPORTANT: We use this, rather than AssetNode.id throughout this file because
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ import {
isSourceAsset,
LiveData,
isHiddenAssetGroupJob,
__ASSET_GROUP_PREFIX,
__ASSET_JOB_PREFIX,
} from '../asset-graph/Utils';
import {AssetGraphQuery_assetNodes} from '../asset-graph/types/AssetGraphQuery';
import {DagsterTypeSummary} from '../dagstertype/DagsterType';
Expand Down Expand Up @@ -232,7 +232,7 @@ const OpNamesDisplay = (props: {

const graphPath = workspacePathFromAddress(
repoAddress,
`/graphs/${__ASSET_GROUP_PREFIX}/${graphName}/`,
`/graphs/${__ASSET_JOB_PREFIX}/${graphName}/`,
);

return (
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_PREFIX} from '../asset-graph/Utils';
import {__ASSET_JOB_PREFIX} from '../asset-graph/Utils';

export interface ExplorerPath {
pipelineName: string;
Expand Down
4 changes: 2 additions & 2 deletions js_modules/dagit/packages/core/src/runs/RunTags.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ import * as React from 'react';

import {SharedToaster} from '../app/DomUtils';
import {useCopyToClipboard} from '../app/browser';
import {__ASSET_GROUP_PREFIX} from '../asset-graph/Utils';
import {__ASSET_JOB_PREFIX} from '../asset-graph/Utils';

import {DagsterTag, RunTag, TagType} from './RunTag';
import {RunFilterToken} from './RunsFilterInput';
Expand Down Expand Up @@ -50,7 +50,7 @@ export const RunTags: React.FC<{
const others = [];
for (const tag of tags) {
if (
tag.value.startsWith(__ASSET_GROUP_PREFIX) &&
tag.value.startsWith(__ASSET_JOB_PREFIX) &&
(tag.key === DagsterTag.PartitionSet || tag.key === DagsterTag.StepSelection)
) {
continue;
Expand Down
9 changes: 5 additions & 4 deletions python_modules/dagster/dagster/core/asset_defs/asset_group.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@
from .assets_job import build_assets_job
from .source_asset import SourceAsset

ASSET_GROUP_BASE_JOB_PREFIX = "__ASSET_GROUP"
# Prefix for auto created jobs that are used to materialize assets
ASSET_BASE_JOB_PREFIX = "__ASSET_JOB"


class AssetGroup:
Expand Down Expand Up @@ -137,7 +138,7 @@ def executor_def(self):

@staticmethod
def is_base_job_name(name) -> bool:
return name.startswith(ASSET_GROUP_BASE_JOB_PREFIX)
return name.startswith(ASSET_BASE_JOB_PREFIX)

def build_job(
self,
Expand Down Expand Up @@ -401,7 +402,7 @@ def get_base_jobs(self) -> Sequence[JobDefinition]:
if len(assets_by_partitions_def.keys()) == 0 or assets_by_partitions_def.keys() == {
None
}:
return [self.build_job(ASSET_GROUP_BASE_JOB_PREFIX)]
return [self.build_job(ASSET_BASE_JOB_PREFIX)]
else:
unpartitioned_assets = assets_by_partitions_def.get(None, [])
jobs = []
Expand All @@ -413,7 +414,7 @@ def get_base_jobs(self) -> Sequence[JobDefinition]:
if partitions_def is not None:
jobs.append(
build_assets_job(
f"{ASSET_GROUP_BASE_JOB_PREFIX}_{i}",
f"{ASSET_BASE_JOB_PREFIX}_{i}",
assets=assets_with_partitions + unpartitioned_assets,
source_assets=[*self.source_assets, *self.assets],
resource_defs=self.resource_defs,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ def test_single_asset_group():
repo_def = repository_def_from_pointer(CodePointer.from_python_file(path, symbol, None))

isinstance(repo_def, RepositoryDefinition)
the_job = repo_def.get_job("__ASSET_GROUP")
the_job = repo_def.get_job("__ASSET_JOB")
assert len(the_job.graph.node_defs) == 2


Expand Down Expand Up @@ -134,7 +134,7 @@ def test_multiple_assets():
repo_def = repository_def_from_pointer(CodePointer.from_python_file(path, symbol, None))

isinstance(repo_def, RepositoryDefinition)
the_job = repo_def.get_job("__ASSET_GROUP")
the_job = repo_def.get_job("__ASSET_JOB")
assert len(the_job.graph.node_defs) == 2


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -997,10 +997,10 @@ def test_job_with_reserved_name():
def the_graph():
pass

the_job = the_graph.to_job(name="__ASSET_GROUP")
the_job = the_graph.to_job(name="__ASSET_JOB")
with pytest.raises(
DagsterInvalidDefinitionError,
match="Attempted to provide job called __ASSET_GROUP to repository, which is a reserved name.",
match="Attempted to provide job called __ASSET_JOB to repository, which is a reserved name.",
):

@repository
Expand Down Expand Up @@ -1109,9 +1109,9 @@ def unpartitioned_asset():
jobs = group.get_base_jobs()
assert len(jobs) == 3
assert {job_def.name for job_def in jobs} == {
"__ASSET_GROUP_0",
"__ASSET_GROUP_1",
"__ASSET_GROUP_2",
"__ASSET_JOB_0",
"__ASSET_JOB_1",
"__ASSET_JOB_2",
}
assert {
frozenset([node_def.name for node_def in job_def.all_node_defs]) for job_def in jobs
Expand Down

0 comments on commit 20f5889

Please sign in to comment.