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

Advanced Params using json-schema #17100

Merged
merged 7 commits into from
Sep 14, 2021
Merged

Advanced Params using json-schema #17100

merged 7 commits into from
Sep 14, 2021

Conversation

msumit
Copy link
Contributor

@msumit msumit commented Jul 20, 2021

Airflow takes a params dictionary at the DAG level or at a Task level that can be overridden by providing dag_run.conf values. However, this params dictionary is quite static in nature and doesn't provide much value addition.

There have been quite some requests made by the community on this already, like #11054, #16430, #17085

Goal

  • Keep the backward compatibility, i.e. the simple params should work as they are right now
  • The params should have the notion of a default value, different types (int, bool, str etc), and various options to validate the user input.
  • The UI should show proper input controls according to the type of param, showing which are must and which are optional, pre-filled with their default values if any.
  • It would be good if UI can show the options list or do live pattern matches if a param utilizes them.
  • Airflow honors these params even if someone triggers a DAG via CLI or API.

Proposal

  • We create a new class or set of classes say Param which can be used in place of the value part of params dictionary.
  • This class should hold the default value and validation rules as well.
  • There should be a method that validates & resolves the value of this Param class. The value could be the default one or provided by the user.
  • We should be able to easily serialize or deserialize it out of DB and use it in place of a normal params value place.
  • Should work with the standard way of DAG creation as well as with the new DAG decorator.

Approaches

pydantic

Pydantic is one of the fastest Python libraries to provide data & type validations (benchmark). I'd implemented various params classes in it (see sample) but did not like the way I had to write validators for each field separately. Also, the order you define fields matters a lot how one can access them in those validator methods.

attrs

Have used attrs previously and it's also in use within Airflow already. attrs simplifies writing classes and also exposes various in-build validators & pre-post init methods. Using attrs it was quite easy to create these classes (see this), though we've to fill in the logic by ourselves to do the data validation. We also felt that more & more such data validation requirements would come from the users and it could turn into a big pile of code in itself.

json-schema

We are using json-schema for DAG serialization already. json-schema has a very powerful & extensive way to define properties (validations) on a field in a language-agnostic way. It has implementation libs in almost all major languages. The custom code using json-schema is pretty minimum (here) & provides very extensive validations.

We should be able to use its Javascript implementation and validate data on the UI itself. The only concern here is that the json-schema rules can become pretty complex easily & users might found it hard to read and understand.

Trigger DAG page

Screenshot 2021-07-15 at 2 06 36 PM

Screenshot 2021-07-15 at 2 08 36 PM

DAG details API

Screenshot 2021-07-15 at 2 10 18 PM

DAG Trigger API

Screenshot 2021-07-15 at 2 11 15 PM

DAG trigger via CLI

$airflow dags trigger example_complex_params --conf '{"str_param": "hello"}'

ValueError: Invalid input for param 'str_param': 'hello' is too long

Failed validating 'maxLength' in schema:
    {'maxLength': 4, 'minLength': 2, 'type': 'string'}

On instance:
    'hello'

Tasks test via CLI

$airflow tasks test example_complex_params all_param 2021-07-15T08:43:45 -t '{"task_param": true}'

ValueError: True is not of type 'string'

Failed validating 'type' in schema:
    {'type': 'string'}

On instance:
    True    

Thanks a lot to @ashb & @kaxil for their inputs.

airflow/models/dag.py Outdated Show resolved Hide resolved
ashb
ashb previously requested changes Aug 3, 2021
Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

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

Broadly looks good, but some main themes to addres:

  • Don't mutate v.default to validate the value
  • Questions about serialization
  • Possibly confusing names with Param and DagParam.

airflow/api_connexion/endpoints/dag_run_endpoint.py Outdated Show resolved Hide resolved
airflow/cli/commands/dag_command.py Outdated Show resolved Hide resolved
airflow/cli/commands/task_command.py Outdated Show resolved Hide resolved
airflow/example_dags/example_complex_params.py Outdated Show resolved Hide resolved
airflow/example_dags/example_complex_params.py Outdated Show resolved Hide resolved
airflow/serialization/serialized_objects.py Outdated Show resolved Hide resolved
airflow/serialization/serialized_objects.py Outdated Show resolved Hide resolved
airflow/www/views.py Outdated Show resolved Hide resolved
tests/models/test_dagparam.py Outdated Show resolved Hide resolved
@msumit msumit force-pushed the params2.0 branch 3 times, most recently from 44b2a6d to 07265cd Compare August 9, 2021 10:44
@ryanahamilton ryanahamilton removed their request for review August 16, 2021 13:40
@kaxil kaxil force-pushed the params2.0 branch 2 times, most recently from bc691a4 to fc0116a Compare August 18, 2021 23:42
@msumit msumit changed the title WIP: Advanced Params using json-schema Advanced Params using json-schema Aug 24, 2021
@paantya
Copy link

paantya commented Aug 30, 2021

see also
#17912

@msumit msumit force-pushed the params2.0 branch 3 times, most recently from 57f5ebd to 452e2d3 Compare September 8, 2021 13:30
Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

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

Looks good -- a couple of small improvements that you can take or leave.

airflow/models/param.py Outdated Show resolved Hide resolved
airflow/models/param.py Outdated Show resolved Hide resolved
airflow/models/param.py Outdated Show resolved Hide resolved
airflow/models/param.py Outdated Show resolved Hide resolved
@github-actions github-actions bot added the full tests needed We need to run full set of tests for this PR to merge label Sep 9, 2021
@github-actions
Copy link

github-actions bot commented Sep 9, 2021

The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.

Copy link
Member

@kaxil kaxil left a comment

Choose a reason for hiding this comment

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

Minor suggestions

airflow/cli/commands/dag_command.py Outdated Show resolved Hide resolved
airflow/cli/commands/task_command.py Outdated Show resolved Hide resolved
Comment on lines 46 to 44
bash_command="echo {{ params.int_param }} {{ params.str_param }} {{ params.old_param }} "
"{{ params.simple_param }} {{ params.email_param }} {{ params.task_param }}",
Copy link
Member

Choose a reason for hiding this comment

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

this is still un-solved

airflow/models/dag.py Outdated Show resolved Hide resolved
airflow/models/param.py Outdated Show resolved Hide resolved
airflow/models/param.py Outdated Show resolved Hide resolved
airflow/models/param.py Show resolved Hide resolved
airflow/serialization/serialized_objects.py Outdated Show resolved Hide resolved
airflow/serialization/serialized_objects.py Outdated Show resolved Hide resolved
tests/models/test_dag.py Outdated Show resolved Hide resolved
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:API Airflow's REST/HTTP API area:CLI area:core-operators Operators, Sensors and hooks within Core Airflow area:serialization area:webserver Webserver related Issues full tests needed We need to run full set of tests for this PR to merge
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants