Skip to content

Commit 5998451

Browse files
authored
Exclude UCX jobs from crawling (#3733)
## Changes Exclude UCX jobs from crawling to avoid confusing for users when they see UCX jobs in their assessment report. ### Linked issues Fixes #3656 Resolves #3722 Follow up on #3732 Relates to #3731 ### Functionality - [x] modified `JobsCrawler` - [x] modified existing workflow: `assessment` ### Tests - [x] added unit tests - [x] added integration tests ### PRs merged into this branch > Merged the following PRs into this branch in an attempt to let the CI pass. Those PRs contain fixes for integration tests From #3767: Scope linted dashboards on mock runtime context. We should use `make_dashboard` instead of the dashboard fixture directly `_make_dashboard`. Also changed one dashboard to a `LakeviewDashboard` so that we lint that too From #3759 Add retry mechanism to wait for the grants to exists before crawling Resolves #3758 - [x] modified integration tests: `test_all_grant_types`
1 parent ef45443 commit 5998451

File tree

9 files changed

+179
-47
lines changed

9 files changed

+179
-47
lines changed

src/databricks/labs/ucx/assessment/jobs.py

Lines changed: 37 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@
88

99
from databricks.labs.lsql.backends import SqlBackend
1010
from databricks.sdk import WorkspaceClient
11+
from databricks.sdk.errors import DatabricksError
1112
from databricks.sdk.service import compute
1213
from databricks.sdk.service.compute import ClusterDetails, ClusterSpec
1314
from databricks.sdk.service.jobs import (
@@ -94,15 +95,49 @@ def _job_clusters(job: BaseJob) -> Iterable[tuple[BaseJob, ClusterSpec]]:
9495

9596

9697
class JobsCrawler(CrawlerBase[JobInfo], JobsMixin, CheckClusterMixin):
98+
"""Crawl jobs (workflows), assess them and store the result in the inventory.
99+
100+
Args :
101+
ws (WorkspaceClient): The workspace client to crawl the jobs with.
102+
sql_backend (SqlBackend): The SQL backend to store the results with.
103+
schema (str): The schema to store the results in.
104+
include_job_ids (list[int] | None): If provided, only include these job ids. Otherwise, include all jobs.
105+
exclude_job_ids (list[int] | None): If provided, exclude these job ids. Otherwise, include all jobs. Note: We
106+
prefer `include_job_ids` for more strict scoping, but sometimes it's easier to exclude a few jobs.
107+
"""
108+
97109
def __init__(
98-
self, ws: WorkspaceClient, sql_backend: SqlBackend, schema, *, include_job_ids: list[int] | None = None
110+
self,
111+
ws: WorkspaceClient,
112+
sql_backend: SqlBackend,
113+
schema,
114+
*,
115+
include_job_ids: list[int] | None = None,
116+
exclude_job_ids: list[int] | None = None,
99117
):
100118
super().__init__(sql_backend, "hive_metastore", schema, "jobs", JobInfo)
101119
self._ws = ws
102120
self._include_job_ids = include_job_ids
121+
self._exclude_job_ids = exclude_job_ids
122+
123+
def _list_jobs(self) -> Iterable[BaseJob]:
124+
"""List the jobs.
125+
126+
If provided, excludes jobs with id in `exclude_job_ids`.
127+
If provided, excludes jobs with id not in `include_job_ids`.
128+
If both provided, `exclude_job_ids` takes precedence.
129+
"""
130+
try:
131+
for job in self._ws.jobs.list(expand_tasks=True):
132+
if self._exclude_job_ids is not None and job.job_id in self._exclude_job_ids:
133+
continue
134+
if self._include_job_ids is None or job.job_id in self._include_job_ids:
135+
yield job
136+
except DatabricksError as e:
137+
logger.error("Cannot list jobs", exc_info=e)
103138

104139
def _crawl(self) -> Iterable[JobInfo]:
105-
all_jobs = list(self._ws.jobs.list(expand_tasks=True))
140+
all_jobs = list(self._list_jobs())
106141
all_clusters = {c.cluster_id: c for c in self._ws.clusters.list() if c.cluster_id}
107142
return self._assess_jobs(all_jobs, all_clusters)
108143

@@ -112,9 +147,6 @@ def _assess_jobs(self, all_jobs: list[BaseJob], all_clusters_by_id) -> Iterable[
112147
job_id = job.job_id
113148
if not job_id:
114149
continue
115-
if self._include_job_ids is not None and job_id not in self._include_job_ids:
116-
logger.info(f"Skipping job_id={job_id}")
117-
continue
118150
cluster_details = ClusterDetails.from_dict(cluster_config.as_dict())
119151
cluster_failures = self._check_cluster_failures(cluster_details, "Job cluster")
120152
cluster_failures.extend(self._check_jar_task(job.settings.tasks))

src/databricks/labs/ucx/contexts/application.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -283,6 +283,7 @@ def jobs_crawler(self) -> JobsCrawler:
283283
self.sql_backend,
284284
self.inventory_database,
285285
include_job_ids=self.config.include_job_ids,
286+
exclude_job_ids=list(self.install_state.jobs.values()),
286287
)
287288

288289
@cached_property

src/databricks/labs/ucx/contexts/workflow_task.py

Lines changed: 1 addition & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
PolicyInfo,
1717
)
1818
from databricks.labs.ucx.assessment.init_scripts import GlobalInitScriptCrawler
19-
from databricks.labs.ucx.assessment.jobs import JobOwnership, JobInfo, JobsCrawler, SubmitRunsCrawler
19+
from databricks.labs.ucx.assessment.jobs import JobOwnership, JobInfo, SubmitRunsCrawler
2020
from databricks.labs.ucx.assessment.pipelines import PipelinesCrawler, PipelineInfo, PipelineOwnership
2121
from databricks.labs.ucx.assessment.sequencing import MigrationSequencer
2222
from databricks.labs.ucx.config import WorkspaceConfig
@@ -68,15 +68,6 @@ def installation(self) -> Installation:
6868
install_folder = self._config_path.parent.as_posix().removeprefix("/Workspace")
6969
return Installation(self.workspace_client, "ucx", install_folder=install_folder)
7070

71-
@cached_property
72-
def jobs_crawler(self) -> JobsCrawler:
73-
return JobsCrawler(
74-
self.workspace_client,
75-
self.sql_backend,
76-
self.inventory_database,
77-
include_job_ids=self.config.include_job_ids,
78-
)
79-
8071
@cached_property
8172
def job_ownership(self) -> JobOwnership:
8273
return JobOwnership(self.administrator_locator)

tests/integration/assessment/test_jobs.py

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,25 @@ def test_job_crawler(ws, make_job, inventory_schema, sql_backend):
3232
assert int(results[0].job_id) == new_job.job_id
3333

3434

35+
def test_job_crawler_excludes_job(ws, make_job, inventory_schema, sql_backend) -> None:
36+
"""Test if the job crawler can exclude a job."""
37+
new_job = make_job(spark_conf=_SPARK_CONF)
38+
skip_job = make_job(spark_conf=_SPARK_CONF)
39+
job_crawler = JobsCrawler(
40+
ws,
41+
sql_backend,
42+
inventory_schema,
43+
# Adding the skip job to the `include_job_ids` scope the crawler from not crawling all jobs while still testing
44+
# the exclude job behaviour
45+
include_job_ids=[new_job.job_id, skip_job.job_id],
46+
exclude_job_ids=[skip_job.job_id],
47+
)
48+
49+
jobs = job_crawler.snapshot()
50+
51+
assert not any(job.job_id == str(skip_job.job_id) for job in jobs)
52+
53+
3554
@retried(on=[NotFound, InvalidParameterValue], timeout=timedelta(minutes=5))
3655
def test_job_run_crawler(ws, env_or_skip, inventory_schema, sql_backend):
3756
cluster_id = env_or_skip("TEST_DEFAULT_CLUSTER_ID")

tests/integration/conftest.py

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -631,10 +631,9 @@ def make_linting_resources(self) -> None:
631631
self.make_job(content="spark.table('old.stuff')")
632632
self.make_job(content="spark.read.parquet('dbfs://mnt/file/')", task_type=SparkPythonTask)
633633
self.make_job(content="spark.table('some.table')", task_type=SparkPythonTask)
634-
query_1 = self.make_query(sql_query='SELECT * from parquet.`dbfs://mnt/foo2/bar2`')
635-
self._make_dashboard(query=query_1)
636-
query_2 = self.make_query(sql_query='SELECT * from my_schema.my_table')
637-
self._make_dashboard(query=query_2)
634+
query_1 = self.make_query(sql_query="SELECT * from parquet.`dbfs://mnt/foo2/bar2`")
635+
self.make_dashboard(query=query_1)
636+
self.make_lakeview_dashboard(query="SELECT * from my_schema.my_table")
638637

639638
def add_table(self, table: TableInfo):
640639
self._tables.append(table)

tests/integration/hive_metastore/test_grant_detail.py

Lines changed: 34 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -1,17 +1,19 @@
11
import json
22
import logging
33
import datetime as dt
4+
from collections.abc import Callable, Iterable
45

56
import pytest
7+
from databricks.labs.lsql.backends import StatementExecutionBackend
68
from databricks.sdk.errors import NotFound
79
from databricks.sdk.retries import retried
810

9-
from databricks.labs.lsql.backends import StatementExecutionBackend
10-
from databricks.labs.ucx.hive_metastore.grants import GrantsCrawler
11+
from databricks.labs.ucx.hive_metastore.grants import Grant, GrantsCrawler
1112
from databricks.labs.ucx.install import deploy_schema
1213

1314
from ..conftest import MockRuntimeContext
1415

16+
1517
logger = logging.getLogger(__name__)
1618

1719

@@ -23,42 +25,52 @@ def _deployed_schema(runtime_ctx) -> None:
2325

2426

2527
@retried(on=[NotFound, TimeoutError], timeout=dt.timedelta(minutes=3))
26-
def test_all_grant_types(
27-
runtime_ctx: MockRuntimeContext, sql_backend: StatementExecutionBackend, _deployed_schema: None
28-
):
29-
"""Test that all types of grants are properly handled by the view when reporting the object type and identifier."""
28+
def test_all_grant_types(runtime_ctx: MockRuntimeContext, _deployed_schema: None):
29+
"""All types of grants should be reported by the grant_detail view."""
3030

3131
# Fixture: a group and schema to hold all the objects, the objects themselves and a grant on each to the group.
3232
group = runtime_ctx.make_group()
3333
schema = runtime_ctx.make_schema()
3434
table = runtime_ctx.make_table(schema_name=schema.name)
3535
view = runtime_ctx.make_table(schema_name=schema.name, view=True, ctas="select 1")
3636
udf = runtime_ctx.make_udf(schema_name=schema.name)
37-
sql_backend.execute(f"GRANT SELECT ON CATALOG {schema.catalog_name} TO `{group.display_name}`")
38-
sql_backend.execute(f"GRANT SELECT ON SCHEMA {schema.full_name} TO `{group.display_name}`")
39-
sql_backend.execute(f"GRANT SELECT ON TABLE {table.full_name} TO `{group.display_name}`")
40-
sql_backend.execute(f"GRANT SELECT ON VIEW {view.full_name} TO `{group.display_name}`")
41-
sql_backend.execute(f"GRANT SELECT ON FUNCTION {udf.full_name} TO `{group.display_name}`")
42-
sql_backend.execute(f"GRANT SELECT ON ANY FILE TO `{group.display_name}`")
43-
sql_backend.execute(f"GRANT SELECT ON ANONYMOUS FUNCTION TO `{group.display_name}`")
37+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON CATALOG {schema.catalog_name} TO `{group.display_name}`")
38+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON SCHEMA {schema.full_name} TO `{group.display_name}`")
39+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON TABLE {table.full_name} TO `{group.display_name}`")
40+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON VIEW {view.full_name} TO `{group.display_name}`")
41+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON FUNCTION {udf.full_name} TO `{group.display_name}`")
42+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON ANY FILE TO `{group.display_name}`")
43+
runtime_ctx.sql_backend.execute(f"GRANT SELECT ON ANONYMOUS FUNCTION TO `{group.display_name}`")
44+
45+
@retried(on=[ValueError], timeout=dt.timedelta(minutes=2))
46+
def wait_for_grants(condition: Callable[[Iterable[Grant]], bool], **kwargs) -> None:
47+
"""Wait for grants to meet the condition.
48+
49+
The method retries the condition check to account for eventual consistency of the permission API.
50+
"""
51+
grants = runtime_ctx.grants_crawler.grants(**kwargs)
52+
if not condition(grants):
53+
raise ValueError("Grants do not meet condition")
4454

45-
# Ensure the view is populated (it's based on the crawled grants) and fetch the content.
46-
GrantsCrawler(runtime_ctx.tables_crawler, runtime_ctx.udfs_crawler).snapshot()
55+
def contains_select_on_any_file(grants: Iterable[Grant]) -> bool:
56+
"""Check if the SELECT permission on ANY FILE is present in the grants."""
57+
return any(g.principal == group.display_name and g.action_type == "SELECT" for g in grants)
58+
59+
# Wait for the grants to be available so that we can snapshot them.
60+
# Only verifying the SELECT permission on ANY FILE as it takes a while to propagate.
61+
wait_for_grants(contains_select_on_any_file, any_file=True)
4762

48-
rows = list(
49-
sql_backend.fetch(
50-
f"""
63+
runtime_ctx.grants_crawler.snapshot()
64+
65+
grants_detail_query = f"""
5166
SELECT object_type, object_id
5267
FROM {runtime_ctx.inventory_database}.grant_detail
5368
WHERE principal_type='group' AND principal='{group.display_name}' and action_type='SELECT'
5469
"""
55-
)
56-
)
57-
grants = {(row.object_type, row.object_id) for row in rows}
70+
grants = {(row.object_type, row.object_id) for row in runtime_ctx.sql_backend.fetch(grants_detail_query)}
5871

5972
# TODO: The types of objects targeted by grants is missclassified; this needs to be fixed.
6073

61-
# Test the results.
6274
expected_grants = {
6375
("TABLE", table.full_name),
6476
("VIEW", view.full_name),

tests/unit/__init__.py

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -165,7 +165,7 @@ def mock_workspace_client(
165165
ws.pipelines.get = _pipeline
166166
ws.workspace.get_status = lambda _: ObjectInfo(object_id=123)
167167
ws.get_workspace_id.return_value = 123
168-
ws.jobs.list.return_value = _id_list(BaseJob, job_ids)
168+
ws.jobs.list.return_value = iter(_id_list(BaseJob, job_ids))
169169
ws.jobs.list_runs.return_value = _id_list(BaseRun, jobruns_ids)
170170
ws.warehouses.get_workspace_warehouse_config().data_access_config = _load_list(EndpointConfPair, warehouse_config)
171171
ws.workspace.export = _workspace_export
@@ -193,6 +193,13 @@ def mock_workspace_client(
193193
{'workspace_id': 789, 'deployment_name': 'test3'},
194194
]
195195
),
196+
'state.json': json.dumps(
197+
{
198+
'resources': {
199+
'jobs': {'test': '123', 'assessment': '456'},
200+
}
201+
}
202+
),
196203
}
197204
ws.workspace.download.side_effect = lambda file_name, *, format=None: io.StringIO(
198205
download_yaml[os.path.basename(file_name)]

tests/unit/assessment/test_jobs.py

Lines changed: 75 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -67,10 +67,12 @@ def test_jobs_assessment_with_spn_cluster_no_job_tasks():
6767

6868
def test_job_crawler_creator():
6969
ws = mock_workspace_client()
70-
ws.jobs.list.return_value = (
71-
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
72-
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
73-
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
70+
ws.jobs.list.return_value = iter(
71+
[
72+
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
73+
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
74+
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
75+
]
7476
)
7577
result = JobsCrawler(ws, MockBackend(), "ucx").snapshot(force_refresh=True)
7678

@@ -80,6 +82,75 @@ def test_job_crawler_creator():
8082
assert set(expected_creators) == set(crawled_creators)
8183

8284

85+
def test_job_crawler_skips_all_jobs_with_empty_include_job_ids() -> None:
86+
"""If `include_job_ids` is empty, all jobs should be skipped."""
87+
ws = mock_workspace_client()
88+
ws.jobs.list.return_value = iter(
89+
[
90+
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
91+
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
92+
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
93+
]
94+
)
95+
96+
result = JobsCrawler(ws, MockBackend(), "ucx", include_job_ids=[]).snapshot(force_refresh=True)
97+
98+
assert not result
99+
100+
101+
def test_job_crawler_include_job_ids() -> None:
102+
"""Only jobs with IDs in `include_job_ids` should be crawled."""
103+
104+
ws = mock_workspace_client()
105+
ws.jobs.list.return_value = iter(
106+
[
107+
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
108+
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
109+
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
110+
]
111+
)
112+
113+
result = JobsCrawler(ws, MockBackend(), "ucx", include_job_ids=[1]).snapshot(force_refresh=True)
114+
115+
assert result == [JobInfo(job_id="1", success=1, failures="[]", job_name="Unknown")]
116+
117+
118+
def test_job_crawler_exclude_job_ids() -> None:
119+
"""The jobs with IDs in `exclude_job_ids` should be skipped."""
120+
121+
ws = mock_workspace_client()
122+
ws.jobs.list.return_value = iter(
123+
[
124+
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
125+
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
126+
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
127+
]
128+
)
129+
130+
result = JobsCrawler(ws, MockBackend(), "ucx", exclude_job_ids=[2, 3]).snapshot(force_refresh=True)
131+
132+
assert result == [JobInfo(job_id="1", success=1, failures="[]", job_name="Unknown")]
133+
134+
135+
def test_job_crawler_exclude_job_ids_takes_preference_over_include_job_ids() -> None:
136+
"""The jobs with IDs in `exclude_job_ids` should be skipped, also when they are in include_job_ids."""
137+
138+
ws = mock_workspace_client()
139+
ws.jobs.list.return_value = iter(
140+
[
141+
BaseJob(job_id=1, settings=JobSettings(), creator_user_name=None),
142+
BaseJob(job_id=2, settings=JobSettings(), creator_user_name=""),
143+
BaseJob(job_id=3, settings=JobSettings(), creator_user_name="bob"),
144+
]
145+
)
146+
147+
result = JobsCrawler(ws, MockBackend(), "ucx", include_job_ids=[1, 2], exclude_job_ids=[2, 3]).snapshot(
148+
force_refresh=True
149+
)
150+
151+
assert result == [JobInfo(job_id="1", success=1, failures="[]", job_name="Unknown")]
152+
153+
83154
@pytest.mark.parametrize(
84155
"jobruns_ids,cluster_ids,run_ids,failures",
85156
[

tests/unit/hive_metastore/test_pipeline_migrate.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -97,7 +97,7 @@ def test_migrate_pipelines_no_pipelines(ws) -> None:
9797
pipelines_crawler = PipelinesCrawler(ws, sql_backend, "inventory_database")
9898
jobs_crawler = JobsCrawler(ws, sql_backend, "inventory_database")
9999
pipelines_migrator = PipelinesMigrator(ws, pipelines_crawler, jobs_crawler, "catalog_name")
100-
ws.jobs.list.return_value = [BaseJob(job_id=536591785949415), BaseJob(), BaseJob(job_id=536591785949417)]
100+
ws.jobs.list.return_value = iter([BaseJob(job_id=536591785949415), BaseJob(), BaseJob(job_id=536591785949417)])
101101
pipelines_migrator.migrate_pipelines()
102102

103103

0 commit comments

Comments
 (0)