apache / airflow

Apache Airflow - A platform to programmatically author, schedule, and monitor workflows
https://airflow.apache.org/
Apache License 2.0
37.12k stars 14.3k forks source link

Make context less nebulous #14396

Closed r-richmond closed 2 years ago

r-richmond commented 3 years ago

Description

Currently context is a dictionary. The dictionary is a bit nebulous especially for new users as it isn't clear what is present inside it and the autocomplete functionality of arbitrary dictionaries leaves a lot to be desired.

Discoverability and type safety could be greatly improved if context was refactored into a dataclass (preferred) or typedDict(less ideal but probably easier).

Use case / motivation

Avoid having to know the keys present in context

def get_current_context() -> Dict:
  pass

context = get_current_context()
ti = context["ti"]

would be instead


@dataclass
class Context:
  # would be the place to see what values are in context and optionally include comments
  ti: TaskInstance # this is the task Instance
  ds: DateTime # This is the execution date
  # etc...

def get_current_context() -> Context:
  pass

context = get_current_context()
#`context.` would have strong autocomplete here
ti = context.ti
Context Dataclass MVP V2 From later in the thread that prevents a breaking change & warns on unsafe updates to context ```python @dataclass class Demo: # context replacement id: str value_dc: int user_defined: Dict[str, Any] = field(default_factory=dict) def __getitem__(self, item): if item in self.__dict__.keys(): logging.warning(msg=f"dictionary interface getitem on context is deprecated; update to use the dataclass interface for standard fields like `{item}`") return self.__dict__[item] elif item in self.user_defined: logging.warning(msg=f"dictionary interface getitem on context is deprecated; update to use context.user_defined for custom fields like `{item}`") return self.user_defined[item] else: raise KeyError def __setitem__(self, key: str, value): if key in self.__dict__.keys(): msg = f"""dictionary interface setitem for standard fields is deprecated; update to use the dataclass interface for standard fields like `{key}` note: changing standard context fields is not supported and may have undefined behavior. If this is meant to be a custom field use context.user_defined instead""" logging.warning(msg=msg) self.__dict__[key] = value else: logging.warning( msg=f"dictionary interface setitem on context is deprecated; update to use context.user_defined for custom fields like `{key}`") self.user_defined[key] = value def keys(self): # added as an example to show how far we could go to have a non-breaking change for 2.1 logging.warning(msg=f"dictionary interface keys is deprecated; update this to use the dataclass interface") temp = self.__dict__ temp.update(self.user_defined) return temp d = Demo(id="long_id", value_dc=1337) print(d["id"]) d["new"] = 3 print(d["new"]) print(d.keys()) d["id"] = "warn" ``` returns ``` WARNING:root:dictionary interface getitem on context is deprecated; update to use the dataclass interface for standard fields like `id` WARNING:root:dictionary interface setitem on context is deprecated; update to use context.user_defined for custom fields like `new` WARNING:root:dictionary interface getitem on context is deprecated; update to use context.user_defined for custom fields like `new` WARNING:root:dictionary interface keys is deprecated; update this to use the dataclass interface WARNING:root:dictionary interface setitem for standard fields is deprecated; update to use the dataclass interface for standard fields like `id` note: changing standard context fields is not supported and may have undefined behavior. If this is meant to be a custom field use context.user_defined instead long_id 3 {'id': 'long_id', 'value_dc': 1337, 'user_defined': {'new': 3}, 'new': 3} ```

My end goal is to be able to do the following when writing callables that can have context passed to them

def generate_is_latest_callable(tasks_if_latest: List[str] , tasks_if_not_latest: List[str]) -> Callable:
  def result(context: Context) -> List[str]:
    context. #and be able to get strong autocomplete and typing while in an ide here
    # because currently all I can do is context: Dict[str, Any] which isn't very helpful
    if context.something:
      return tasks_if_latest
    else:
      return tasks_if_not_latest
  return result

t_branch = PythonBranchingOperator(
  task_id="branch",
  python_callable=generate_is_latest_callable(["yes"], ["no"]),
  provide_context=True,
  dag=dag,
)

Are you willing to submit a PR?

This one touches too much of airflow's internals for me to try and tackle.

Related Issues

Not to my knowledge Slack Conversation: https://apache-airflow.slack.com/archives/C0146STM600/p1614103441043600?thread_ts=1614099549.042800&cid=C0146STM600

@kaxil

potiuk commented 3 years ago

I think context as a ductionary is pretty sound. Dataclass is not pythonic at all and makes it difficult to evolve. I think it's very pythonic to have dictionaries in Python and it brings a number of benefits, with little drawback

r-richmond commented 3 years ago

Dataclass is not pythonic at all

As a counter point Dacite is a python library that shows how dataclasses can be used for something like this. Their config dataclass is a similar object to our context and is defined as follows.

https://github.com/konradhalas/dacite/blob/d2206b2e4711859da0ea5862c395940f33693e80/dacite/config.py#L5-L12

@dataclass
class Config:
    type_hooks: Dict[Type, Callable[[Any], Any]] = field(default_factory=dict)
    cast: List[Type] = field(default_factory=list)
    forward_references: Optional[Dict[str, Any]] = None
    check_types: bool = True
    strict: bool = False
    strict_unions_match: bool = False

which is then consumed in their function from_dict https://github.com/konradhalas/dacite/blob/d2206b2e4711859da0ea5862c395940f33693e80/dacite/core.py#L34

def from_dict(data_class: Type[T], data: Data, config: Optional[Config] = None) -> T:
    """Create a data class instance from a dictionary.
    :param data_class: a data class type
    :param data: a dictionary of a input data
    :param config: a configuration of the creation process
    :return: an instance of a data class

As a user it feels so much better to be able to see the config in the function definition and then follow the type definition straight to the config objects strict definition (fields, types, optionally comments).

Alternatively when a dictionary is used its harder for an end user to reason about what keys are present?, what are the values?, what are the types of the values?, where do I find this information?. (Perhaps its my own ignorance but it isn't obvious to me today where I can look up this info for airflow's context dictionary and I've tried).

As a developer tools like mypy, pydantic & various ides (pycharm, vscode) also handle dataclasses quite well which helps prevent frustrating runtime errors.

makes it difficult to evolve

Not sure about this one.

Perhaps I'm still a rookie but for dictionaries it feels much harder to be sure you've gotten all the old key references out of the code, before you can safely remove it.

I think it's very pythonic to have dictionaries in Python and it brings a number of benefits, with little drawback

I've called out a couple advantages I think dataclasses have over dictionaries. But I'm likely missing the dictionary advantages. Is Airflow using any of these advantages today?

kaxil commented 3 years ago

We could also add a TypedDict -- at least IDEs will have good auto-completion

potiuk commented 3 years ago

We could also add a TypedDict -- at least IDEs will have good auto-completion

Yep. @kaxil is right - TypedDict is far more pythonic (Hints are the way how types are usually added in Python). The thing with using fixed Dataclasses. We should basically never remove/rename anything in the context (maybe we should use some deprecation mechanism).

The context is used by pretty much every single custom operator out there, and the change you propose will immediately break all of them.. If anything else - this is a single reason why we should not do it.

But the idea of improving developer's productivity is sound and since we have TypedDict which is backwards compatible and gives most benefits you mentioned, I think it is far better solution. Would youl like to pick this up @r-richmond ?

r-richmond commented 3 years ago

TypedDict is far more pythonic (Hints are the way how types are usually added in Python)

Hints are how dataclasses uses types as well. They aren't enforced at runtime, unless you use something like pydantic. (side note Dataclass PEP-557 had support from Raymond & Guido which as far as I can tell gives it a lot of points in the pythonic category. Also Raymond gave a fantastic talk on Dataclasses that I'd recommend if you have time).

We should basically never remove/rename anything in the context (maybe we should use some deprecation mechanism).

Yep makes sense to me. Although, this isn't a point for or against dataclasses, typedDicts, or dicts.

The context is used by pretty much every single custom operator out there, and the change you propose will immediately break all of them.. If anything else - this is a single reason why we should not do it.

While switching to dataclasses would be harder than TypedDicts it does not have to be a breaking change. You can convert dictionaries to dataclasses and vice versa pretty seamlessly. Although, implementing Dataclasses would certainly be more work to get right in the initial pr.

Example showing how you can switch between dataclasses & dictionaries ```Python from dataclasses import dataclass, asdict from dacite import from_dict @dataclass class Demo: id: str value: int d = Demo("a", 1) examples = [d, asdict(d), from_dict(data_class=Demo, data=asdict(d))] for example in examples: print(example, type(example)) ``` prints ``` Demo(id='a', value=1) {'id': 'a', 'value': 1} Demo(id='a', value=1) ```

Personally I'm of the opinion that working with the dataclass api is superior to working with the dictionary api due to the reasons outlined above but thats just like my opinion man.

But the idea of improving developer's productivity is sound and since we have TypedDict which is backwards compatible and gives most benefits you mentioned

Either option could be backwards compatible but I agree that the TypedDict change is more straight forward and is an alternative, hence the tiny blurb about it in the opener.

As you mentioned TypedDict should give most of the benefits, although to my knowledge find references does not work as well dictionary keys which is a bummer. Anyways the decision on which to use is above my Airflow status so I'll let the maintainers decide. Most importantly I think either a TypedDict or Dataclass implementation would be a big improvement over the current situation.

Would youl like to pick this up @r-richmond ?

Actually, the reason I submitted this issue was because me and a coworker were frustrated with the "magic context dictionary". TLDR I'm not actually sure what all the keys, values, & types are in context so I wouldn't even know how to start this one.

My end goal is to be able to do the following when writing callable that can have context passed to them ```Python def generate_is_latest_callable(tasks_if_latest: List[str] , tasks_if_not_latest: List[str]) -> Callable: def result(context: Context) -> List[str]: context. #and be able to get strong autocomplete and typing while in an ide here # because currently all I can do is context: Dict[str, Any] which isn't very helpful if context.something: return tasks_if_latest else: return tasks_if_not_latest return result t_branch = PythonBranchingOperator( task_id="branch", python_callable=generate_is_latest_callable(["yes"], ["no"]), provide_context=True, dag=dag, ) ``` Note: moving this goal to the opening comment
kaxil commented 3 years ago

Oh yeah, I will add docs (in few mins / hours) to make the context dictionary a bit less magical, it contains the following items:

https://github.com/apache/airflow/blob/352b970010846eb4168aa7b2e332d9a95e2facb3/airflow/models/taskinstance.py#L1688-L1728

r-richmond commented 3 years ago

Ah interesting kaxil thanks to your link I can see where a TypedDict / Dataclass implementation would go

https://github.com/apache/airflow/blob/352b970010846eb4168aa7b2e332d9a95e2facb3/airflow/models/taskinstance.py#L86

Since the TypedDict implementation is so straightforward given that link I wonder if its better to consider Dict > TypeDict > DataClass as an evolution path.

potiuk commented 3 years ago

While switching to dataclasses would be harder than TypedDicts it does not have to be a breaking change. You can convert dictionaries to dataclasses and vice versa pretty seamlessly. Although, implementing Dataclasses would certainly be more work to get right in the initial pr.

Let me explain why this would be a breaking change. if you have an operator that already uses context. Example from task decorator case (but this is the same for any custom operator):

from airflow.operators.python import task, get_current_context

@task
def my_task():
    context = get_current_context()
    ti = context["ti"]

Surely you can map from Dataclass to Dict in "get_current_context()" method (otherwise you'd have to also correct the context["ti"] into context.ti. Thus it would be a breaking change.

But if you do that (i.e map dataclass to dict), then you loose the type hinting. This is precisely the place where typehinting is needed (in the user code). And I think Typed Dict (if used everywhere including return value from get_current_context() gives you what you need from the user point of view and there is no need to implement the > Dataclass step - as you have everything you need by implementing TypeDict.

Or maybe I am missing something?

Personally I'm of the opinion that working with the dataclass api is superior to working with the dictionary api due to the reasons outlined above but thats just like my opinion man.

In a number of cases yes. But in others flexibility of the dict and fact that you can add any value there, trumps.

potiuk commented 3 years ago

Also one more point - since we are in Python world - users could do whatever they want in the Python code of their custom operators. Currently 'context' is part of the public API of Airflow that users rely on being dictionary.

For example I can very easily imagine someone decides to add a new field to context and pass it further.

@task
def my_task():
    context = get_current_context()
    ti = context["ti"]
    context['my_key'] = 'my_value'
    call_another_function(context)

While we do not encourage this behavior in any way, it is tempting to use the context this way and as a user you could do that and you could rely on context being dictionary able to hold any key you want. Dataclass breaks this assumption.

r-richmond commented 3 years ago

Let me explain why this would be a breaking change.... Surely you can map from Dataclass to Dict in "get_current_context()" method (otherwise you'd have to also correct the context["ti"] into context.ti. Thus it would be a breaking change.

Changing `get_current_context` to have the following signature would prevent any breaking changes ```python def get_current_context(return_dataclass: bool=False) -> Union[Dict[str, Any], ~Dataclass.Context]: context: Dataclass = # if return_dataclass: return context else: logging.warning(msg="dictionary context has been deprecated in 2.1 please use return_dataclass = True") return as_dict(context) ``` existing code continues just fine ```python @task def my_task(): context = get_current_context() ti = context["ti"] ``` this would then allow the users to do the following to get type hinting and at some point we could update the option's default value before finally removing it. ```python @task def my_task(return_dataclass=True): context = get_current_context() ti = context.ti ```

But if you do that (i.e map dataclass to dict), then you loose the type hinting. This is precisely the place where typehinting is needed (in the user code). And I think Typed Dict (if used everywhere including return value from get_current_context() gives you what you need from the user point of view and there is no need to implement the > Dataclass step - as you have everything you need by implementing TypeDict.

You're right that the returned dictionary appears to lose the type information. However:

  1. No type information is what we have today in user code so no change (although we'd have a nice clear definition of context findable from the function signature).
  2. Users would have an easy way to switch to strict typing mode with a simple migration path one function call at a time via the parameter.
    3. We would also give them the ability to use the nicer dataclass api & better IDE support Find references with dataclass (shows writes and reads and highlights field names) image vs Find references with TypedDict (Misses read and doesn't highlight field names) image

  1. Users would also have a clear definition of context (This comes with either TypedDict or Dataclass) just wanted to call out this improvement.

The advantage to dataclasses is that long term we will be providing users with a better api and developer experience. Making this change now is the easiest it will ever be as it only gets harder later in the project's life cycle.

Even in the Typed Dict Pep 589 Dataclasses are called out as a newer alternative for this use case. Dataclasses are a more recent alternative to solve this use case

In a number of cases yes. But in others flexibility of the dict and fact that you can add any value there, trumps.

While we do not encourage this behavior in any way, it is tempting to use the context this way and as a user you could do that and you could rely on context being dictionary able to hold any key you want. Dataclass breaks this assumption.

You can do this with dataclasses as well; although as you mentioned it isn't encouraged. The better way would be to subclass the Dataclass but that is up to end users.

Demo of adding a field dynamically ```python @dataclass class Demo: id: str value: int d = Demo("a", 1) # can add new fields d.new_field = "hello world" print(d.new_field) # prints hello world ``` One nice thing is that mypy complains about it however the python code still runs `main.py:21: error: "Demo" has no attribute "new_field"`

I'm unaware of anything a dictionary can do that a dataclass can't. (I believe dataclasses internally are just dictionaries like a lot of python internals)

Or maybe I am missing something?

The only thing I'd guess that you are missing is maybe how nice dataclasses are :p . On the other hand I'm sure I'm missing some complexity about switching to dataclasses.

To summarize here either dataclass or typedDict will have working code but leaning on Raymond here in my assessment it feels like context is a class/instance problem not a dict problem hence my personal preference to use a dataclass.

potiuk commented 3 years ago

Seems like we end up with ever compiicating interfaces for no benefit other than "newer" approach where we have simpler alternative.

Now, let me see how would you like to implement backward-compatibility of this (this is the more standard case of custom operator. This is just copy of what we have in our documentation:

from airflow.models.baseoperator import BaseOperator
from airflow.utils.decorators import apply_defaults

class TaskOperator(BaseOperator):

    @apply_defaults
    def __init__(
            self,
            name: str,
            **kwargs) -> None:
        super().__init__(**kwargs)
        self.name = name

    def execute(self, context):
        message = "Task {}".format(context['ti'])
        print(message)
        return message

What solution would be good here if we use Dataclasses, we want to keep backwards compatibility and we want to give users typehint for context.

Any idea how to do it short of converting the Dataclass to TypeDict before calling execute (which would defeat the purpose of having Dataclass in the first place).?

r-richmond commented 3 years ago

Seems like we end up with ever compiicating interfaces for no benefit other than "newer" approach where we have simpler alternative.

I'd argue that the dataclass interface is clearer than the dictionary interface and thus simpler to use, although perhaps not as simple to author.

Now, let me see how would you like to implement backward-compatibility of this (this is the more standard case of custom operator. This is just copy of what we have in our documentation:

What solution would be good here if we use Dataclasses, we want to keep backwards compatibility and we want to give users typehint for context.

Any idea how to do it short of converting the Dataclass to TypeDict before calling execute (which would defeat the purpose of having Dataclass in the first place).?

First off great challenge :) , 2nd I'd probably replace my earlier suggestion with the following as well


from dataclasses import dataclass, asdict

@dataclass
class Demo:  # context replacement
    id: str
    value_dc: int

    def __getitem__(self, item):
        logging.warning(msg=f"dictionary interface is deprecated please update this to use the dataclass interface")
        return asdict(self)[item]

    # I don't know if the following methods are needed but are here just to demonstrate how far we could go
    # to make the change non-breaking
    def items(self):
        logging.warning(msg=f"dictionary interface is deprecated please update this to use the dataclass interface")
        return asdict(self).items()

    def keys(self):
        logging.warning(msg=f"dictionary interface is deprecated please update this to use the dataclass interface")
        return asdict(self).keys()

    def values(self):
        logging.warning(msg=f"dictionary interface is deprecated please update this to use the dataclass interface")
        return asdict(self).values()

d = Demo(id="long_id", value_dc=1337)
print(d["id"])

returns

WARNING:root:dictionary interface is deprecated please update this to use the dataclass interface
long_id

Seems like this would allow us to go straight to dataclasses without changing any functions or operators anywhere. We could then remove the dictionary api at the next major version.

What do you think? Is there another use case that I'm not aware of that this won't work for?

potiuk commented 3 years ago

First off great challenge :) , 2nd I'd probably replace my earlier suggestion with the following as well

Yeah. I was expecting exactly this answer :). So summarizing - what you've done now, you created hybrid dataclass and dictionary put together. Now, my goal is to show you what further consequences you have to deal with.

First of all this "hybrid" solution will be around for years. We will not get rid of it in 2.* - we have to wait for 3.0.0 release (which we did not even start thinking about). We started rigorously following SEMVER and we cannot remove any "public API" behaviour till 3.0. Context is probably the most "public" API of Airflow you can imagine.. There are hundreds of thousands custom operators our there that are using this "public API". This means that any change we introduce now is going to be around at least a year from now. And until then 1.10 is still there as well (and will be there for quite some time). So if someone develops custom operators for their 1.10 Airflow, they will still use dictionary - so we have probably tens of thousands custom operators created still using the 'Dictionary' context for another year or two.

This is as a bit of context (!). I try to think empathically about our users. As someone who cares not only about 'purity' of the implemenation I have to think also about the adoption of Airlfow and years of maintenance.

I am sure you realize that, but the example I've shown you above is the simplest form. Context is shared in a few places with custom operators:

Most likely many of the currently released operators are using the context to pass data (as custom dictionary values) between those methods - one can set a custom value in pre_execute() and retrieve it in execute() or post_execute() reads whatever execute sets in the context. It was easy to use, we have not forbidden it, it is part of the API (this is the basic "property" of dictionary - unlike dataclass - that you can set any value with any key there). By introducing Dataclass we are breaking this property. You will not be able to set arbitrary key in the context in pre_execute so that it is available in execute. If we implement the interrim (lasting at least a year or more) hybrid dataclass <-> dictionary proposed above, this will continue to work but with deprecation warnings.

If we decide this is the right way to go, we have to communicate it to the users, and prepare them for the migration and give them the tools necessary to migrate when it comes to 3.0.0. Our goal is (as it was in 2.0.0 migration) to make it as smooth as possible for the migration. We even developed upgrade_check script that users can run on their installation which will tell them what needs to be fixed: https://airflow.apache.org/docs/apache-airflow/stable/upgrade-check.html

I imagine the same will be with 3.0.0. I imagine, that we prepare another upgrade check. Imagine all the different scenarios people will have by then: 1) Custom operators in DAGs 2) Custom operators as Plugins 3) Custom operators installed as custom providers (this has been added in 2.0 - we want to encourage people to build their own providers and distribute them as PyPI packages). 4) Localy installed custom operators as part of python environment/images users will have.

So my challenge for you is this - propose a strategy that will help to migrate all those cases by our users - in the way that will not make them refrain from migration to 3.0.0 with the Dataclass context. What checks are you going to use, how are you going to encourage the users to migrate all the thousands of DAGs they have, what tools you are going to provide them.

And once you do that, I will ask you - and other committers (undoubtedly looking at that conversation) - to answer single question:

kaxil commented 3 years ago

I agree, let's use TypedDict for now -- We have added a cross-reference in docs so hopefully it is a bit clearer than before:

http://apache-airflow-docs.s3-website.eu-central-1.amazonaws.com/docs/apache-airflow/latest/concepts.html#accessing-current-context

http://apache-airflow-docs.s3-website.eu-central-1.amazonaws.com/docs/apache-airflow/latest/macros-ref.html (It contains a nice table with docs to show what users can use).

kaxil commented 3 years ago

Also remember since users can use custom Macros, the DataClass or TypedDict approach will help for the inbuilt context -- it won't help for custom macros

r-richmond commented 3 years ago

Yeah. I was expecting exactly this answer :). So summarizing - what you've done now, you created hybrid dataclass and dictionary put together. Now, my goal is to show you what further consequences you have to deal with.

0_O

There are hundreds of thousands custom operators our there that are using this "public API". This means that any change we introduce now is going to be around at least a year from now. And until then 1.10 is still there as well (and will be there for quite some time). So if someone develops custom operators for their 1.10 Airflow, they will still use dictionary - so we have probably tens of thousands custom operators created still using the 'Dictionary' context for another year or two.

Yes, this is true. Thats why I mentioned that now is the "easiest" time for this change as it only gets harder in the future; (In a perfect world, this would have been raised before 2.0 :s).

Most likely many of the currently released operators are using the context to pass data (as custom dictionary values) between those methods - one can set a custom value in pre_execute() and retrieve it in execute() or post_execute() reads whatever execute sets in the context. It was easy to use, we have not forbidden it, it is part of the API (this is the basic "property" of dictionary - unlike dataclass - that you can set any value with any key there). By introducing Dataclass we are breaking this property. You will not be able to set arbitrary key in the context in pre_execute so that it is available in execute. If we implement the interrim (lasting at least a year or more) hybrid dataclass <-> dictionary proposed above, this will continue to work but with deprecation warnings.

Again dataclass does not break this property and in fact after thinking, I think dataclasses provides the following significant advantages in this area which dict and typedDict do not.

  1. A safer way for users to set custom fields in context
  2. A safer way for airflow maintainers to add new fields to context

Take the following example

Context Dataclass MVP V2 ```python @dataclass class Demo: # context replacement id: str value_dc: int user_defined: Dict[str, Any] = field(default_factory=dict) def __getitem__(self, item): if item in self.__dict__.keys(): logging.warning(msg=f"dictionary interface getitem on context is deprecated; update to use the dataclass interface for standard fields like `{item}`") return self.__dict__[item] elif item in self.user_defined: logging.warning(msg=f"dictionary interface getitem on context is deprecated; update to use context.user_defined for custom fields like `{item}`") return self.user_defined[item] else: raise KeyError def __setitem__(self, key: str, value): if key in self.__dict__.keys(): msg = f"""dictionary interface setitem for standard fields is deprecated; update to use the dataclass interface for standard fields like `{key}` note: changing standard context fields is not supported and may have undefined behavior. If this is meant to be a custom field use context.user_defined instead""" logging.warning(msg=msg) self.__dict__[key] = value else: logging.warning( msg=f"dictionary interface setitem on context is deprecated; update to use context.user_defined for custom fields like `{key}`") self.user_defined[key] = value def keys(self): # added as an example to show how far we could go to have a non-breaking change for 2.1 logging.warning(msg=f"dictionary interface keys is deprecated; update this to use the dataclass interface") temp = self.__dict__ temp.update(self.user_defined) return temp d = Demo(id="long_id", value_dc=1337) print(d["id"]) d["new"] = 3 print(d["new"]) print(d.keys()) d["id"] = "warn" ``` returns ``` WARNING:root:dictionary interface getitem on context is deprecated; update to use the dataclass interface for standard fields like `id` WARNING:root:dictionary interface setitem on context is deprecated; update to use context.user_defined for custom fields like `new` WARNING:root:dictionary interface getitem on context is deprecated; update to use context.user_defined for custom fields like `new` WARNING:root:dictionary interface keys is deprecated; update this to use the dataclass interface WARNING:root:dictionary interface setitem for standard fields is deprecated; update to use the dataclass interface for standard fields like `id` note: changing standard context fields is not supported and may have undefined behavior. If this is meant to be a custom field use context.user_defined instead long_id 3 {'id': 'long_id', 'value_dc': 1337, 'user_defined': {'new': 3}, 'new': 3} ```

By using the dataclass Airflow is now able to do several things

  1. Provide a safe place for users to store user_defined context attributes without risk of clobbering current keys.
    • what happens if we need to add a key to context in the future that some of these 1000's of operators are now using as a custom field? (I believe they'd have broken code until they tracked down the silent error due to the context key re-write)
  2. Provide clear and helpful deprecation and warning notices for bugs that people may have today and not realize
    • i.e. what happens if today someone is overwrite a key today without realizing it
    • More word smithing would probably be needed on the sample warnings above
  3. We can let users subclass context if they want as an alternative to using user_defined dict to get strong typing.
  4. We are able to provide a non-breaking change implementation for 2.1

So my challenge for you is this - propose a strategy that will help to migrate all those cases by our users - in the way that will not make them refrain from migration to 3.0.0 with the Dataclass context. What checks are you going to use, how are you going to encourage the users to migrate all the thousands of DAGs they have, what tools you are going to provide them.

My strategy is to add this deprecation warning as soon as possible i.e. 2.1 and give the users until 3.0 to update to the new interface. This will reduce the number of operators that get created with the old dictionary interface and give users the most time to upgrade. Additionally a quick 1 pager upgrade guide could be written to show the change in interface.

And once you do that, I will ask you - and other committers (undoubtedly looking at that conversation) - to answer single question: Is it worth the hassle if we can achieve the very same user experience by using TypedDict ?

The thing is you don't get the same user experience by using TypedDict. By using dataclasses you get the following additional benefits.

  1. Better IDE support (find usages/ refactoring / name highlighting) for users and mainters (see above for image example)
  2. A safer interface for users who want to add custom fields to context
  3. A safer way for maintainers to add new fields to context
  4. A clean way to implement deprecation warnings with detailed warning messages about potential silent bugs
  5. More flexibility down the road (dataclasses are more flexible than dictionaries)
  6. A solution that is easier to maintain in the future
    • I concede that right now and for the rest of 2.x it will be more complicated to maintain. However thinking about the long-term future of airflow I think once Airflow 3+ rolls around we will be happy to have made this change due the reasons outlined above.
    • Or said another way we shouldn't optimize for airflow 2.x maintainability we should optimize for airflow maintainability.

p.s. sorry for another wall of text. I guess it turns out that I'm a little passionate on this one...

potiuk commented 3 years ago

Better IDE support (find usages/ refactoring / name highlighting) for users and mainters (see above for image example) There won't by any renames for those. That would further break compatibility. so making it "easier" is false sense of security when you rename

A safer interface for users who want to add custom fields to context

Not really. We are not planning to add anything to the context any time soon. and even if we do it's the same for dict/field. If someone extends data class with a new field the problem is the same.

A safer way for maintainers to add new fields to context

Not really. it's the same kind of problems you get.

A clean way to implement deprecation warnings with detailed warning messages about potential silent bugs

We do not need deprecation warnings in case we do not change from Dict

More flexibility down the road (dataclasses are more flexible than dictionaries)

This sentence is meaningless. I argue that dicts are more flexible and probably we would both be right.

A solution that is easier to maintain in the future

Again - meaningless - maintenance is also to go trough the hassle of changing and informing users.

Or said another way we shouldn't optimize for airflow 2.x maintainability we should optimize for airflow maintainability.

I do not agree. I carefully weighted pros/cons and as maintainer i agree with @kaxil TypedDict is much better solution and we will have no plans to change to Dataclass. You have not convinced us.

p.s. sorry for another wall of text. I guess it turns out that I'm a little passionate on this one...

No problems with being passionate (I am very passionate myself as some other committers might attest to), but I think it's good to realise that passion might easily turn into obsession and be able to say 'meh'.

r-richmond commented 3 years ago

Not really. We are not planning to add anything to the context any time soon. and even if we do it's the same for dict/field. If someone extends data class with a new field the problem is the same.

A safer way for maintainers to add new fields to context Not really. it's the same kind of problems you get.

I don't follow here. In the V2 dataclass design users would add stuff to the user_defined field which has no chance of collision with anything airflow adds later to the base dataclass. The same can't be said for dicts. What am I missing?

A clean way to implement deprecation warnings with detailed warning messages about potential silent bugs We do not need deprecation warnings in case we do not change from Dict

True regarding the deprecation warning, but isn't their value in warning users about doing things could cause bugs? (i.e. overwriting the wrong keys in context).

More flexibility down the road (dataclasses are more flexible than dictionaries) This sentence is meaningless. I argue that dicts are more flexible and probably we would both be right.

I like the meet in the middle vibe but I kind of cheated with the user_defined field being a dict which I think pushes dataclass to the winners column.

A solution that is easier to maintain in the future Again - meaningless - maintenance is also to go trough the hassle of changing and informing users.

I think we are just stuck on 2 different points here, you rightly point to the burden of migrating users and I'm stuck on what I perceive are the long-term benefits of switching (safer modifications, better IDE integration usability improvements, warning users about things that could cause bugs).

Or said another way we shouldn't optimize for airflow 2.x maintainability we should optimize for airflow maintainability. I do not agree. I carefully weighted pros/cons and as maintainer i agree with @kaxil TypedDict is much better solution and we will have no plans to change to Dataclass. You have not convinced us.

I'm just a user so apologies if I came across as forceful. It is clear we have a difference of opinion in which case the maintainer usually and rightly decides upon the path forward for the project.

Lastly at this point I'm gonna slow down on this topic for a while to allow other users & maintainers a chance to give their opinion/thoughts.

r-richmond commented 3 years ago

@kaxil Noticed this keeps getting pushed. Can you provide any additional context?

potiuk commented 3 years ago

I believe so far simply no-one else found it really useful or needed. We usually do not plan work which is not "urgent", "important" or "part of bigger feature we work on".

Actually if you would like to convince others that this is useful and helpful - why don't you create a Draft PR with it and share it here to see how it can look like so that you could reiterate your points showing them with a real code. If you convince others (including myself), we will merge it. If not - we will drop it.

kaxil commented 3 years ago

Exactly what Jarek said

uranusjr commented 3 years ago

Maybe a typing.TypedDict definition would be useful though. It requires much less change (basically just write a type definition and use it in some function annotations) and do mostly the same thing for the user if they have type hints set up.

r-richmond commented 3 years ago

I believe so far simply no-one else found it really useful or needed.

What is the bar to clear here?.. i.e. 5 thumbs up on opening issue doesn't meet it so I'm curious at what point your opinion would change.

We usually do not plan work which is not "urgent", "important" or "part of bigger feature we work on".

Makes sense, ty for info

why don't you create a Draft PR with it and share it here to see how it can look like so that you could reiterate your points showing them with a real code. If you convince others (including myself), we will merge it. If not - we will drop it.

I'm interested in seeing it tackled and am willing to take a stab at it. I'm just really busy for the next couple months but will give it a go after that. I.e. please don't close the issue yet.

kaxil commented 3 years ago

Yup we would not close it for sure :) . It's just that it hasn't fallen into priorities as yet.

Anyone who is willing to work on the issue should take a stab at it and we are happy to help if needed.

Even if there are no thumbs up, you can still work on it :)

potiuk commented 3 years ago

100% what @kaxil wrote. There is no "bar" to clean (and we are not going to close it).

I was merely answering your question @r-richmond - you asked for the additional context so I did:

Can you provide any additional context?

The more elaborated version of the context is - in open-source project if no-one is interested in working on the issue, it will not get implemented. It's as simple as that.

So answer "why" things are pushed back is very simple - it's not pushed back. Simply NOONE (including maintainers) is finding it worthy to work on it. And it does not mean that the issue is invalid - it just means it's either not urgent enough or not important enough to spend our time on it. If you now the "urgent/not-urgent" vs. " important/not-important" matrix - it means it falls into "not-urgent and not-important". There plenty of other issues that fall into the 3 other squares in the matrix. And this "matrix" is different for different individuals - i.e. it can be "urgent" or "important" for various resons for YOU, but apparently it's not for most other people.

So if YOU think it's either important or urgent, then the best way to get it done (if no-one else does not think the same) is that YOU take a lead and implement it.

And if it's not gonna be commented or worked on for quite some time it will be closed automatically - because that will mean that even the author lost interest in it.

It's reallly really as simple and straightforward as this. There is no politics, there is no-one to decide which issue we are working or not (except that every one of the people involved - either committer or contributor) makes individual decision "this is what I am going to work on next" .

potiuk commented 3 years ago

BTW. If you have no time to implement it yourself, then the second-best approach is to convince others that it is either "urgent" or "important" or "both". But this is totally and entirely up to the power of your arguments (and sometimes it can take much more effort and time to convince others to implement than to implement it yourself and THEN convince others).

ferruzzi commented 3 years ago

Looking at maybe taking this on. I also had a good bit of "what the heck is even IN context and what's it for?" when I first started looking at the codebase, so I can see the benefit to new contributors and it'll be a good one to learn, myself.

1) Would the ideal solution be to make a new module under airflow.models to define the Context TypedDict and import it into taskinstance.py, or just define it at the top of airflow.models.taskinstance?

2) If/once I do define the TypedDict, are any other changes needed in order to implement it? I have a rough implementation locally already and it seems to pass CI without any further changes, but want to make sure I'm not underestimating the scope of this.

3) Context appears to contain some other Dicts (conf and var, specifically), should I chase that rabbit and define those as well while I am at it?

4) It looks like we use both datetime.datetime and pendulum.DateTime in various places, which would we prefer to use here in the Context definition for the various timestamp fields? [[ Discussing here ]]

potiuk commented 3 years ago

Looking at maybe taking this on. I also had a good bit of "what the heck is even IN context and what's it for?" when I first started looking at the codebase, so I can see the benefit to new contributors and it'll be a good one to learn, myself.

Cool!

I think we have to make sure that we know what can be achieved here and what the "real" benefits of context stronger naming and typing is. Just adding typying without being able to make good use of it is kinda useless. I.e. where and for whom actually "more typing and autocompletable naming" in context would help.

Let me summarize my view on what the requirements here are and who are the "users" of each requirement. What we are talking about here is to make easier to "write" dags and a bit easier to read them as context.ti is much more readable than context.['ti']. We could also thing about some kind of verification and flagging typos.

  1. Would the ideal solution be to make a new module under airflow.models to define the Context TypedDict and import it into taskinstance.py, or just define it at the top of airflow.models.taskinstance?

I am not sure whether TypedDict is the right solution here, if we look at the above list. I'd rather say that a dedicated Context class which mixes-in the dict and field access would be a better solution in this case. When it comes to docstrings and explanation of the fields, and auto-complete in @task decorator, I am not sure TypedDict is a good solution. But that needs some exploring I think.

  1. If/once I do define the TypedDict, are any other changes needed in order to implement it? I have a rough implementation locally already and it seems to pass CI without any further changes, but want to make sure I'm not underestimating the scope of this.

As above - I am not sure how many users will benefit from "TypedDict" approach. I belive making a dedicated Context class with mix-in like approach woudl serve more cases from the list above.

  1. Context appears to contain some other Dicts (conf and var, specifically), should I chase that rabbit and define those as well while I am at it?
  1. It looks like we use both datetime.datetime and pendulum.DateTime in various places, which would we prefer to use here in the Context definition for the various timestamp fields? [[ Discussing here ]]

I think pendulum.DateTime (https://airflow.apache.org/docs/apache-airflow/stable/templates-ref.html?highlight=next_ds#variables). I think we should look at the code and see what kind of "datetime" we have there already. But that is one of the very good side-effects of this change - maybe MyPy will be able to detect when we mix the datetime's and warn us here?

ferruzzi commented 3 years ago

@potiuk Extracting a Context class out into a new module was my initial thought, but it looked like you were arguing pretty hard for the TypedDict solution. A new airflow.models.context module won't be a trivial change and I want to make sure we're "all" on the same page before wasting a ton of time on something that will get shot down based on the implementation.

Looking at the TaskInstance module, there are a few challenges with this solution. The module itself is a bit of a mess right now (I'm getting over 75 IDE warnings/errors as it stands), Context is pretty heavily entwined in it, and there would maybe/likely be a circular dependency between the TaskInstance, Context, and BaseOperator modules to sort out. This could turn into a pretty major surgery, so I'd feel better about taking it on after we have a bit of consensus that it's the path we want to go down.

uranusjr commented 3 years ago

I don’t see how it would require a major surgery. A TypedDict is just a dict a runtime, and type checking is designed to be able to handle circular type dependencies (see PEP 484 on Forward Referneces).

potiuk commented 3 years ago

@potiuk Extracting a Context class out into a new module was my initial thought, but it looked like you were arguing pretty hard for the TypedDict solution. A new airflow.models.context module won't be a trivial change and I want to make sure we're "all" on the same page before wasting a ton of time on something that will get shot down based on the implementation.

I just got more on the front of "what we want to get from it" rather than "how to implement it" (maybe I grew up a bit since the last time we discussed and look at it more from a "product" point of view rather than implementation detail. I am not sure if TypedDict will be enough to support all the use cases above - but since it's about the time you are implementing it, it might be worth looking at alternatives - because maybe different aproach will yeild more natural use by the users. I think we are changing it now precisely because we wan to make it easier for the users of "context" to get more natural experience and simply have it easier to use.

So whatever solution we come up with - it's just the question of how benefital it will be for the users. I am not against or for TypedDict or separate class - I just think that when we are touching it - we should see whether we can make it easier to use in those cases I listed above :). Let's just think about those cases and see how the solution we come up fits-in and which of the cases it supports..

ferruzzi commented 3 years ago

As a starting point, I've moved Context and two helper methods into their own module. It took some shenanigans to get around the TaskInstance/Context/BaseOperator circular import, and !! THIS DOES NOT BUILD !! but how do we feel about the general direction it's going?

https://github.com/ferruzzi/airflow/commit/4c86dbbab968a09386d09380a80650b5383bf5e4

uranusjr commented 3 years ago

Looks good to me in general. I think you can avoid a lot of the workarounds by keeping get_template_context on TaskInstance instead of making it a standalone function. That function always needs to be called with a TaskInstance anyway.

potiuk commented 2 years ago

Yep. It does look good.

Looks good to me in general. I think you can avoid a lot of the workarounds by keeping get_template_context on TaskInstance instead of making it a standalone function. That function always needs to be called with a TaskInstance anyway.

I rather like the way @ferruzzi did it - look how many less imports the TaskInstance class has this way. For me ti really makes perfect sense to tie the "get_template_context" with Context in separate module and keep the imports there - they were only needed there to build "context fields" which are not actually used by TaskInstance. I like it much better.

ferruzzi commented 2 years ago

@uranusjr - I do see what you are saying. Passing ti=self to get_context_template like that feels a little off. I went this route because it feels like a better logical fit in the Context module since it's essentially a Factory/Builder method. The three modules are pretty heavily inter-related so I don't know if there is going to be a really pretty and clean solution, but I will keep trying to untangle it.

I just wanted to get a temperature check before spending too much time running in the wrong direction. It sounds like we're all comfortable with the direction so far. As it stands right now, when I run the CI tests locally all import errors are clear, which is a good sign, but many/most/all of the WWW tests time out saying failed on setup with "Failed: Timeout >60.0s". So that's where I'm at with it at the moment.

potiuk commented 2 years ago

I just wanted to get a temperature check before spending too much time running in the wrong direction. It sounds like we're all comfortable with the direction so far. As it stands right now, when I run the CI tests locally all import errors are clear, which is a good sign, but many/most/all of the WWW tests time out saying failed on setup with "Failed: Timeout >60.0s". So that's where I'm at with it at the moment.

We are about to fix some last "main" tests failing - so maybe wait until that is done and rebase to see if this is not related @ferruzzi

ferruzzi commented 2 years ago

Sounds good. I saw that Breeze broke recently, and I'm also having hardware issues so there were a lot of potential causes there. I'll give you some time and come back to it. Cheers.

potiuk commented 2 years ago

Sounds good. I saw that Breeze broke recently, and I'm also having hardware issues so there were a lot of potential causes there. I'll give you some time and come back to it. Cheers.

That was Javascript failing not Breeze :). But yeah last few days were not the greenest

ferruzzi commented 2 years ago

Following conversion on that PR, the solution that uranusjr has going is better than mine, so I'll step back on this one and let them wrap it up.

potiuk commented 2 years ago

I think it's long addressed.