Skip to content

Commit

Permalink
Feature flags to let dagit vary features for certain code locations (#…
Browse files Browse the repository at this point in the history
…17016)

Summary:
For users on versions that are able to use thew new backfill policy, we
want to hide the backfill dialog from the backfill page. For users still
on older versions, we want to leave it. This gives the frontend the
tools to check that.

Test Plan: BK

## Summary & Motivation

## How I Tested These Changes
  • Loading branch information
gibsondan authored and benpankow committed Oct 9, 2023
1 parent a2b6bab commit bc266f8
Show file tree
Hide file tree
Showing 6 changed files with 196 additions and 3 deletions.

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

22 changes: 22 additions & 0 deletions js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts

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

18 changes: 18 additions & 0 deletions python_modules/dagster-graphql/dagster_graphql/schema/external.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
GrpcServerCodeLocation,
ManagedGrpcPythonEnvCodeLocationOrigin,
)
from dagster._core.host_representation.feature_flags import get_feature_flags_for_location
from dagster._core.host_representation.grpc_server_state_subscriber import (
LocationStateChangeEvent,
LocationStateChangeEventType,
Expand Down Expand Up @@ -163,6 +164,14 @@ class Meta:
name = "WorkspaceLocationStatusEntriesOrError"


class GrapheneFeatureFlag(graphene.ObjectType):
class Meta:
name = "FeatureFlag"

name = graphene.NonNull(graphene.String)
enabled = graphene.NonNull(graphene.Boolean)


class GrapheneWorkspaceLocationEntry(graphene.ObjectType):
id = graphene.NonNull(graphene.ID)
name = graphene.NonNull(graphene.String)
Expand All @@ -173,6 +182,8 @@ class GrapheneWorkspaceLocationEntry(graphene.ObjectType):

permissions = graphene.Field(non_null_list(GraphenePermission))

featureFlags = non_null_list(GrapheneFeatureFlag)

class Meta:
name = "WorkspaceLocationEntry"

Expand Down Expand Up @@ -210,6 +221,13 @@ def resolve_permissions(self, graphene_info):
permissions = graphene_info.context.permissions_for_location(location_name=self.name)
return [GraphenePermission(permission, value) for permission, value in permissions.items()]

def resolve_featureFlags(self, graphene_info):
feature_flags = get_feature_flags_for_location(self._location_entry)
return [
GrapheneFeatureFlag(name=feature_flag_name.value, enabled=feature_flag_enabled)
for feature_flag_name, feature_flag_enabled in feature_flags.items()
]


class GrapheneRepository(graphene.ObjectType):
id = graphene.NonNull(graphene.ID)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,9 @@

from dagster import file_relative_path
from dagster._core.host_representation import ManagedGrpcPythonEnvCodeLocationOrigin
from dagster._core.host_representation.feature_flags import (
CodeLocationFeatureFlags,
)
from dagster._core.types.loadable_target_origin import LoadableTargetOrigin
from dagster._core.workspace.load import location_origins_from_yaml_paths
from dagster.version import __version__ as dagster_version
Expand Down Expand Up @@ -46,6 +49,10 @@
value
}
updatedTimestamp
featureFlags {
name
enabled
}
}
}
... on PythonError {
Expand Down Expand Up @@ -149,6 +156,13 @@ def test_load_workspace(self, graphql_context):
metadatas = node["displayMetadata"]
metadata_dict = {metadata["key"]: metadata["value"] for metadata in metadatas}

feature_flags = node["featureFlags"]
assert len(feature_flags) == 1
assert (
feature_flags[0]["name"]
== CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE.value
)

assert (
"python_file" in metadata_dict
or "module_name" in metadata_dict
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
from enum import Enum
from typing import TYPE_CHECKING, Mapping

import packaging.version

if TYPE_CHECKING:
from dagster._core.workspace.workspace import CodeLocationEntry


class CodeLocationFeatureFlags(Enum):
SHOW_SINGLE_RUN_BACKFILL_TOGGLE = "SHOW_SINGLE_RUN_BACKFILL_TOGGLE"


def get_feature_flags_for_location(
code_location_entry: "CodeLocationEntry",
) -> Mapping[CodeLocationFeatureFlags, bool]:
return {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: (
get_should_show_single_run_backfill_toggle(code_location_entry)
)
}


def get_should_show_single_run_backfill_toggle(code_location_entry: "CodeLocationEntry"):
# Starting in version 1.5 we stopped showing the single-run backfill toggle in the UI -
# instead it is now set in code

if not code_location_entry.code_location:
# Error or loading status
return False

dagster_library_version = (
code_location_entry.code_location.get_dagster_library_versions() or {}
).get("dagster")

if not dagster_library_version:
# Old enough version that it wasn't being stored
return True

if dagster_library_version == "1!0+dev":
return False

try:
version = packaging.version.parse(dagster_library_version)
return version.major < 1 or (version.major == 1 and version.minor < 5)
except packaging.version.InvalidVersion:
return False
Original file line number Diff line number Diff line change
@@ -1,11 +1,16 @@
import time
from typing import Mapping
from unittest import mock

import pytest
from dagster._core.errors import (
DagsterCodeLocationLoadError,
DagsterCodeLocationNotFoundError,
)
from dagster._core.host_representation.feature_flags import (
CodeLocationFeatureFlags,
get_feature_flags_for_location,
)
from dagster._core.host_representation.origin import RegisteredCodeLocationOrigin
from dagster._core.workspace.context import WorkspaceRequestContext
from dagster._core.workspace.workspace import (
Expand All @@ -15,12 +20,13 @@
from dagster._utils.error import SerializableErrorInfo


def test_get_code_location():
@pytest.fixture
def workspace_request_context():
mock_loc = mock.MagicMock()

error_info = SerializableErrorInfo(message="oopsie", stack=[], cls_name="Exception")

context = WorkspaceRequestContext(
return WorkspaceRequestContext(
instance=mock.MagicMock(),
workspace_snapshot={
"loading_loc": CodeLocationEntry(
Expand Down Expand Up @@ -54,7 +60,10 @@ def test_get_code_location():
read_only=True,
)

assert context.get_code_location("loaded_loc") == mock_loc

def test_get_code_location(workspace_request_context):
context = workspace_request_context
assert context.get_code_location("loaded_loc")
with pytest.raises(DagsterCodeLocationLoadError, match="oopsie"):
context.get_code_location("error_loc")

Expand All @@ -68,3 +77,80 @@ def test_get_code_location():
match="Location missing_loc does not exist in workspace",
):
context.get_code_location("missing_loc")


def _location_with_mocked_versions(dagster_library_versions: Mapping[str, str]):
code_location = mock.MagicMock()
code_location.get_dagster_library_versions = mock.MagicMock(
return_value=dagster_library_versions
)

return CodeLocationEntry(
origin=RegisteredCodeLocationOrigin("loaded_loc"),
code_location=code_location,
load_error=None,
load_status=CodeLocationLoadStatus.LOADED,
display_metadata={},
update_timestamp=time.time(),
)


def test_feature_flags(workspace_request_context):
workspace_snapshot = workspace_request_context.get_workspace_snapshot()

error_loc = workspace_snapshot["error_loc"]
assert get_feature_flags_for_location(error_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

loading_loc = workspace_snapshot["loading_loc"]

assert get_feature_flags_for_location(loading_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

# Old version that didn't even have it set
really_old_version_loc = _location_with_mocked_versions({})

assert get_feature_flags_for_location(really_old_version_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: True
}

# old pre 1.5.0 version
pre_10_version_loc = _location_with_mocked_versions({"dagster": "0.15.5"})

assert get_feature_flags_for_location(pre_10_version_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: True
}

# old pre 1.5.0 version
old_version_loc = _location_with_mocked_versions({"dagster": "1.4.5"})

assert get_feature_flags_for_location(old_version_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: True
}

# Post 1.5.0 version
new_version_loc = _location_with_mocked_versions({"dagster": "1.5.0"})

assert get_feature_flags_for_location(new_version_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

future_version_loc = _location_with_mocked_versions({"dagster": "2.5.0"})

assert get_feature_flags_for_location(future_version_loc) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

gibberish_version = _location_with_mocked_versions({"dagster": "BLAHBLAHBLAH"})

assert get_feature_flags_for_location(gibberish_version) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

dev_version = _location_with_mocked_versions({"dagster": "1!0+dev"})

assert get_feature_flags_for_location(dev_version) == {
CodeLocationFeatureFlags.SHOW_SINGLE_RUN_BACKFILL_TOGGLE: False
}

0 comments on commit bc266f8

Please sign in to comment.