Skip to content

Commit

Permalink
fix compute kind tag (#22504)
Browse files Browse the repository at this point in the history
## Summary & Motivation

#22359 caused the op snapshots created on the code server and loaded in
a host process to have different snapshot IDs, due to patching in of the
new compute kind tag in the host process. This causes a snapshot ID
mismatch error in plus. This PR:

- Adds a backcompat test to ensure the snapshot ID for an op doesn't
change.
- Removes the snap layer host process patch in favor of a GQL layer
patch for ops.
- Patches UI to look for either the legacy or new tag when looking at
asset tags. This is necessary because these tags are not patched at the
GQL layer (they should be in a followup, and we can remove all reference
to the old tag in the UI)

## How I Tested These Changes

New test
  • Loading branch information
smackesey committed Jun 12, 2024
1 parent eba3855 commit e731c93
Show file tree
Hide file tree
Showing 6 changed files with 31 additions and 15 deletions.
10 changes: 8 additions & 2 deletions js_modules/dagster-ui/packages/ui-core/src/graph/KindTags.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,14 @@ import {
linkToAssetTableWithStorageKindFilter,
} from '../search/useGlobalSearch';

export const isCanonicalComputeKindTag = (tag: DefinitionTag) => tag.key === 'dagster/compute_kind';
export const isCanonicalStorageKindTag = (tag: DefinitionTag) => tag.key === 'dagster/storage_kind';
export const LEGACY_COMPUTE_KIND_TAG = 'kind';
export const COMPUTE_KIND_TAG = 'dagster/compute_kind';
export const STORAGE_KIND_TAG = 'dagster/storage_kind';

// Older code servers may be using the legacy compute kind tag, so we need to check for both
export const isCanonicalComputeKindTag = (tag: DefinitionTag) =>
tag.key === COMPUTE_KIND_TAG || tag.key === LEGACY_COMPUTE_KIND_TAG;
export const isCanonicalStorageKindTag = (tag: DefinitionTag) => tag.key === STORAGE_KIND_TAG;
export const buildStorageKindTag = (storageKind: string): DefinitionTag =>
buildDefinitionTag({key: 'dagster/storage_kind', value: storageKind});

Expand Down
3 changes: 2 additions & 1 deletion js_modules/dagster-ui/packages/ui-core/src/graph/OpNode.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ import {Colors, FontFamily, Icon} from '@dagster-io/ui-components';
import * as React from 'react';
import styled from 'styled-components';

import {COMPUTE_KIND_TAG} from './KindTags';
import {OpIOBox, metadataForIO} from './OpIOBox';
import {IOpTag, OpTags} from './OpTags';
import {OpLayout} from './asyncGraphLayout';
Expand Down Expand Up @@ -99,7 +100,7 @@ export class OpNode extends React.Component<IOpNodeProps> {

const tags: IOpTag[] = [];

const kind = metadata.find((m) => m.key === 'kind');
const kind = metadata.find((m) => m.key === COMPUTE_KIND_TAG);
const composite = definition.__typename === 'CompositeSolidDefinition';

if (kind) {
Expand Down
17 changes: 13 additions & 4 deletions python_modules/dagster-graphql/dagster_graphql/schema/solids.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
from dagster._core.snap import DependencyStructureIndex, GraphDefSnap, OpDefSnap
from dagster._core.snap.node import InputMappingSnap, OutputMappingSnap
from dagster._core.storage.dagster_run import RunsFilter
from dagster._core.storage.tags import COMPUTE_KIND_TAG, LEGACY_COMPUTE_KIND_TAG

from dagster_graphql.implementation.asset_checks_loader import AssetChecksLoader
from dagster_graphql.implementation.events import iterate_metadata_entries
Expand Down Expand Up @@ -385,10 +386,18 @@ def __init__(self, represented_pipeline: RepresentedJob, solid_def_name: str):
self._solid_def_snap = represented_pipeline.get_node_def_snap(solid_def_name)

def resolve_metadata(self, _graphene_info):
return [
GrapheneMetadataItemDefinition(key=item[0], value=item[1])
for item in self._solid_def_snap.tags.items()
]
metadata_items = []
for key, val in self._solid_def_snap.tags.items():
metadata_items.append(GrapheneMetadataItemDefinition(key=key, value=val))
# Backcompat for legacy system tags. Older code servers may report the compute kind under
# the legacy tag. Code servers running versions after deprecation of the legacy tag may
# have both the legacy and current tag set. We can't patch this at the Snap level
# because the host process will complain about mismatched snap IDs
if key == LEGACY_COMPUTE_KIND_TAG and COMPUTE_KIND_TAG not in self._solid_def_snap.tags:
metadata_items.append(
GrapheneMetadataItemDefinition(key=COMPUTE_KIND_TAG, value=val)
)
return metadata_items

@property
def solid_def_name(self) -> str:
Expand Down
7 changes: 0 additions & 7 deletions python_modules/dagster/dagster/_core/snap/node.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
MetadataValue,
normalize_metadata,
)
from dagster._core.storage.tags import COMPUTE_KIND_TAG, LEGACY_COMPUTE_KIND_TAG
from dagster._serdes import whitelist_for_serdes

from .dep_snapshot import DependencyStructureSnapshot, build_dep_structure_snapshot_from_graph_def
Expand Down Expand Up @@ -263,12 +262,6 @@ def __new__(
required_resource_keys: Sequence[str],
config_field_snap: Optional[ConfigFieldSnap],
):
# Backcompat for legacy system tags. Older code servers may report the compute kind under
# the legacy tag. Code servers running versions after deprecation of the legacy tag may
# have both the legacy and current tag set.
if LEGACY_COMPUTE_KIND_TAG in tags and COMPUTE_KIND_TAG not in tags:
tags = {COMPUTE_KIND_TAG: tags[LEGACY_COMPUTE_KIND_TAG], **tags}

return super(OpDefSnap, cls).__new__(
cls,
required_resource_keys=check.sequence_param(
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
from dagster._core.instance import DagsterInstance, InstanceRef
from dagster._core.remote_representation.external_data import ExternalStaticPartitionsDefinitionData
from dagster._core.scheduler.instigation import InstigatorState, InstigatorTick
from dagster._core.snap.job_snapshot import JobSnapshot
from dagster._core.storage.dagster_run import DagsterRun, DagsterRunStatus, RunsFilter
from dagster._core.storage.event_log.migration import migrate_event_log_data
from dagster._core.storage.event_log.sql_event_log import SqlEventLogStorage
Expand Down Expand Up @@ -1296,7 +1297,7 @@ def test_known_execution_state_step_output_version_serialization() -> None:
assert deserialize_value(serialized, KnownExecutionState) == known_state


def test_legacy_compute_kind_tag_backcompat():
def test_legacy_compute_kind_tag_backcompat() -> None:
legacy_tags = {LEGACY_COMPUTE_KIND_TAG: "foo"}
with pytest.warns(DeprecationWarning, match="Legacy compute kind tag"):

Expand All @@ -1313,3 +1314,9 @@ def legacy_op():
pass

assert legacy_op.tags[COMPUTE_KIND_TAG] == "foo"

legacy_snap_path = file_relative_path(__file__, "1_7_9_kind_op_job_snap.gz")
legacy_snap = deserialize_value(
GzipFile(legacy_snap_path, mode="r").read().decode("utf-8"), JobSnapshot
)
assert create_snapshot_id(legacy_snap) == "8db90f128b7eaa5c229bdde372e39d5cbecdc7e4"

1 comment on commit e731c93

@github-actions
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Deploy preview for dagit-core-storybook ready!

✅ Preview
https://dagit-core-storybook-cljhc7dtu-elementl.vercel.app

Built with commit e731c93.
This pull request is being automatically deployed with vercel-action

Please sign in to comment.