Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[AIRFLOW-6425] Serialization: Add missing DAG parameters to Json Schema #7002

Merged
merged 2 commits into from
Jan 2, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion airflow/models/dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -1513,7 +1513,9 @@ def get_serialized_fields(cls):
'parent_dag', '_old_context_manager_dags', 'safe_dag_id', 'last_loaded',
'_full_filepath', 'user_defined_filters', 'user_defined_macros',
'_schedule_interval', 'partial', '_old_context_manager_dags',
'_pickle_id', '_log', 'is_subdag', 'task_dict'
'_pickle_id', '_log', 'is_subdag', 'task_dict', 'template_searchpath',
'sla_miss_callback', 'on_success_callback', 'on_failure_callback',
'template_undefined', 'jinja_environment_kwargs'
}
return cls.__serialized_fields

Expand Down
12 changes: 10 additions & 2 deletions airflow/serialization/json_schema.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,9 @@ def iter_errors(self, instance) -> Iterable[jsonschema.exceptions.ValidationErro
...


def load_dag_schema() -> Validator:
def load_dag_schema_dict() -> dict:
"""
Load Json Schema for DAG
Load & return Json Schema for DAG as Python dict
"""
schema_file_name = 'schema.json'
schema_file = pkgutil.get_data(__name__, schema_file_name)
Expand All @@ -61,5 +61,13 @@ def load_dag_schema() -> Validator:
raise AirflowException("Schema file {} does not exists".format(schema_file_name))

schema = json.loads(schema_file.decode())
return schema


def load_dag_schema() -> Validator:
"""
Load & Validate Json Schema for DAG
"""
schema = load_dag_schema_dict()
jsonschema.Draft7Validator.check_schema(schema)
return jsonschema.Draft7Validator(schema)
5 changes: 4 additions & 1 deletion airflow/serialization/schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,10 @@
"start_date": { "$ref": "#/definitions/datetime" },
"end_date": { "$ref": "#/definitions/datetime" },
"dagrun_timeout": { "$ref": "#/definitions/timedelta" },
"doc_md": { "type" : "string"}
"doc_md": { "type" : "string"},
"_default_view": { "type" : "string"},
"_access_control": {"$ref": "#/definitions/dict" },
"is_paused_upon_creation": { "type": "boolean" }
},
"required": [
"params",
Expand Down
15 changes: 15 additions & 0 deletions tests/serialization/test_dag_serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
from airflow.models.baseoperator import BaseOperator
from airflow.operators.bash_operator import BashOperator
from airflow.operators.subdag_operator import SubDagOperator
from airflow.serialization.json_schema import load_dag_schema_dict
from airflow.serialization.serialized_objects import SerializedBaseOperator, SerializedDAG
from tests.test_utils.mock_operators import CustomOperator, CustomOpLink, GoogleLink

Expand All @@ -53,6 +54,7 @@
}
},
"start_date": 1564617600.0,
"is_paused_upon_creation": False,
"params": {},
"_dag_id": "simple_dag",
"fileloc": None,
Expand Down Expand Up @@ -107,6 +109,7 @@ def make_simple_dag():
"depends_on_past": False,
},
start_date=datetime(2019, 8, 1),
is_paused_upon_creation=False,
)
BaseOperator(task_id='simple_task', dag=dag, owner='airflow')
CustomOperator(task_id='custom_task', dag=dag)
Expand Down Expand Up @@ -462,6 +465,18 @@ def test_extra_serialized_field_and_multiple_operator_links(self):
google_link_from_plugin = simple_task.get_extra_links(test_date, GoogleLink.name)
self.assertEqual("https://www.google.com", google_link_from_plugin)

def test_dag_serialized_fields_with_schema(self):
"""
Additional Properties are disabled on DAGs. This test verifies that all the
keys in DAG.get_serialized_fields are listed in Schema definition.
"""
dag_schema: dict = load_dag_schema_dict()["definitions"]["dag"]["properties"]

# The parameters we add manually in Serialization needs to be ignored
ignored_keys: set = {"is_subdag", "tasks"}
dag_params: set = set(dag_schema.keys()) - ignored_keys
self.assertEqual(set(DAG.get_serialized_fields()), dag_params)


if __name__ == '__main__':
unittest.main()