Regarding the warning, maybe a PendingDeprecationWarning would be appropriate? 
It is arguably not too useful in general, but does somewhat express the 
intention here.

--
Sent from my iPhone

> On 17 Sep 2024, at 19:05, Ash Berlin-Taylor <a...@apache.org> wrote:
> 
> Coming back to this after the Summit, I have refined my thoughts some.
> 
> As a reminder, The things we currently allow to be imported from the 
> top-level Airflow module is this:
> 
> https://github.com/apache/airflow/blob/8fe286108720ba1f9a303c066022349d6656f735/airflow/__init__.py#L77-L83
> 
> ```
>    "DAG": (".models.dag", "DAG", False),
>    "Dataset": (".datasets", "Dataset", False),
>    "XComArg": (".models.xcom_arg", "XComArg", False),
> ```
> 
> It seems like there’s loose consensus on airflow.sdk as the module path.
> 
> I’m now thinking that `from airflow import DAG` should continue to work 
> without a deprecation warning, even though in my talk about this very subject 
> I said it would I am now reconsidering that. While we could have this issue a 
> deprecation warning pointing to `from airflow.sdk import DAG` this feels like 
> a massive change to DAG authors for, and feels now to me like a change for 
> change's sake, so what I’m proposing is this:
> 
> I propose that we continue to allow the three top level imports without 
> issuing deprecation warnings (other than `Dataset`, which I think will issue 
> a warning as it is being renamed to Asset?)
> 
> Everything else user facing will be imported from somewhere under 
> `airflow.sdk`, possibly directly on that module (even if internally it is 
> defined elsewhere in the sdk dist.)
> 
> For example, this would work without deprecation:
> 
> 
> ```
> from airflow import DAG, XComArg
> from airflow.sdk import task, TaskGroup, Asset
> ```
> 
> And then airflow/sdk/__init__.py would look something like:
> 
> ```
> from airflow.sdk.definitions.dag import DAG as DAG
> from airflow.sdk.definitions.task_group import TaskGroup as TaskGroup
> …
> ```
> 
> etc (or maybe it could be lazy loaded too. Not relevant to this discussion)
> 
> I think that we should re-write all the docs to use `from airflow.sdk import 
> DAG` as the new “canonical” way, but don’t have it issue a deprecation 
> warning in 3.0, and wait for that until ~3.3 or 3.4.
> 
> Thoughts?
> 
> 
> 
>> On 3 Sep 2024, at 19:24, Fritz Davenport <fr...@astronomer.io.INVALID> wrote:
>> 
>> To add another DAG Author perspective, I'd vote for:
>> #1 (from airflow ...) but without side-effects
>> or #2 (from airflow.sdk ...).
>> 
>> To compare with other X-as-code tools:
>> - Luigi has top-level *Luigi.task* (is class-based)
>> - Prefect has top-level *from prefect import flow, task* (and seems to
>> refer to it as an sdk <https://docs-3.prefect.io/3.0/api-ref/index>)
>> - dagster has top-level *from dagster import asset*
>> - pyspark doesn't have top-level, but uses specific names (e.g. *from
>> pyspark.sql import SparkSession*)
>> - pulumi seems to be top-level, kinda *from pulumi_<provider> import
>> <resource>*
>> 
>>> On Tue, Sep 3, 2024 at 9:15 AM Julian LaNeve <jul...@astronomer.io.invalid>
>>> wrote:
>>> 
>>> Chiming in here mostly from the DAG author perspective!
>>> 
>>> I like `airflow.sdk` best. It makes it super clear what the user is
>>> supposed to interact with and what Airflow’s “public” interface is.
>>> Importing from `airflow.models` has always felt weird because it feels like
>>> you’re going into Airflow’s internals, and importing from things like
>>> `airflow.utils` just added to the confusion because it was always super
>>> unclear what a normal user is supposed to interact with vs what’s internal
>>> and subject to change.
>>> 
>>> The only slight downside (imo) to `airflow.sdk` is that an SDK is
>>> traditionally used to manage/interact with APIs (e.g. the Stripe SDK), so
>>> you could make the case that an “Airflow SDK” should be a library to
>>> interact with Airflow’s API. We’ve run into this before with Astro, where
>>> we published the Astro SDK as an Airflow provider for doing ETL. Then we
>>> were considering releasing a separate tool for interacting with Astro’s API
>>> (creating deployments, etc), which we would’ve called an “Astro SDK” but
>>> that name was already taken. I don’t think we’ll run into that here because
>>> we already have the `clients` concept to interact with the API.
>>> 
>>> The `airflow.definitions` pattern feels odd because it’s not something
>>> I’ve seen elsewhere, so a user would have to learn/remember the pattern
>>> just for Airflow. The top level option also feels nice but the “user” of
>>> Airflow is more than just a DAG author, so I wouldn’t want to restrict
>>> top-level imports just to one audience.
>>> 
>>> --
>>> Julian LaNeve
>>> CTO
>>> 
>>> Email: jul...@astronomer.io
>>> <mailto:jul...@astronomer.io>Mobile: 330 509 5792
>>> 
>>>> On Sep 2, 2024, at 6:46 AM, Jarek Potiuk <ja...@potiuk.com> wrote:
>>>> 
>>>> Yep so. If we do not have side-effects from import airflow -> my vote
>>> would
>>>> be "airflow.sdk" :)
>>>> 
>>>> On Mon, Sep 2, 2024 at 10:29 AM Ash Berlin-Taylor <a...@apache.org>
>>> wrote:
>>>> 
>>>>> Yes, strongly agreed on the “no side-effects form `import airflow`”.
>>>>> 
>>>>> To summarise the options so far:
>>>>> 
>>>>> 1. `from airflow import DAG, TaskGroup` — have the imports be from the
>>> top
>>>>> level airflow module
>>>>> 2. `from airflow.definitions import DAG, TaskGroup`
>>>>> 3. `from airflow.sdk import DAG, TaskGroup`
>>>>> 
>>>>>> On 31 Aug 2024, at 23:07, Jarek Potiuk <ja...@potiuk.com> wrote:
>>>>>> 
>>>>>> Should be:
>>>>>> 
>>>>>> ```
>>>>>> @configure_settings
>>>>>> @configure_worker_plugins
>>>>>> def cli_worker():
>>>>>> pass
>>>>>> ```
>>>>>> 
>>>>>> On Sun, Sep 1, 2024 at 12:05 AM Jarek Potiuk <ja...@potiuk.com> wrote:
>>>>>> 
>>>>>>> Personally for me "airflow.sdk" is best and very straightforward. And
>>> we
>>>>>>> have not yet used that for other things before, so it's free to use.
>>>>>>> 
>>>>>>> "Models" and similar carried more (often misleading) information -
>>> they
>>>>>>> were sometimes database models, sometimes they were not. This caused a
>>>>> lot
>>>>>>> of confusion.
>>>>>>> 
>>>>>>> IMHO explicitly calling something "sdk" is a clear indication "this is
>>>>>>> what you are expected to use". And makes it very clear what is and
>>> what
>>>>> is
>>>>>>> not a public interface. We should aim to make everything in
>>>>> "airflow.<sdk>"
>>>>>>> (or whatever we choose) "public" and everything else "private". That
>>>>> should
>>>>>>> also reduce the need of having to have a separate description of "what
>>>>> is
>>>>>>> public and what is not".
>>>>>>> 
>>>>>>> Actually - if we continue doing import initialization as we do today
>>> - I
>>>>>>> would even go as far as the "airflow_sdk" package - unless we do
>>>>> something
>>>>>>> else that we have had a problem with for a long time - getting rid of
>>>>> side
>>>>>>> effects of "airflow" import.
>>>>>>> 
>>>>>>> It's a bit tangential but actually related - as part of this work we
>>>>>>> should IMHO get rid of all side-effects of "import airflow" that we
>>>>>>> currently have. If we stick to sub-package of airflow  - it is almost
>>> a
>>>>>>> given thing since "airflow.sdk"  (or whatever we choose) will be
>>>>>>> available to "worker", "dag file processor" and "triggerer" but the
>>>>> rest of
>>>>>>> the "airlfow","whatever" will not be, and they won't be able to use
>>> DB,
>>>>>>> where scheduler, api_server will.
>>>>>>> 
>>>>>>> So having side effects - such as connecting to the DB, configuring
>>>>>>> settings, plugin manager initialization when you do "import" caused a
>>>>> lot
>>>>>>> of pain, cyclic imports and a number of other problems.
>>>>>>> 
>>>>>>> I think we should aim to  make "initialization" code explicit rather
>>>>> than
>>>>>>> implicit (Python zen) - and (possibly via decorators) simply
>>> initialize
>>>>>>> what is needed and in the right sequence explicitly for each command.
>>>>> If we
>>>>>>> will be able to do it "airflow.sdk" is ok, if we will still have
>>> "import
>>>>>>> airflow" side-effects, The "airflow_sdk" (or similar) is in this case
>>>>>>> better, because otherwise we will have to have some ugly conditional
>>>>> code -
>>>>>>> when you have and when you do not have database access.
>>>>>>> 
>>>>>>> As an example - If we go for "airflow.sdk" I'd love to see something
>>>>> like
>>>>>>> that:
>>>>>>> 
>>>>>>> ```
>>>>>>> @configure_db
>>>>>>> @configure_settings
>>>>>>> def cli_db():
>>>>>>> pass
>>>>>>> 
>>>>>>> @configure_db
>>>>>>> @configure_settings
>>>>>>> @configure_ui_plugins
>>>>>>> def cli_webserver():
>>>>>>> pass
>>>>>>> 
>>>>>>> @configure_settings
>>>>>>> @configure_ui_plugins
>>>>>>> def cli_worker():
>>>>>>> pass
>>>>>>> ```
>>>>>>> 
>>>>>>> Rather than that:
>>>>>>> 
>>>>>>> ```
>>>>>>> import airflow <-- here everything gets initialized
>>>>>>> ```
>>>>>>> 
>>>>>>> J
>>>>>>> 
>>>>>>> 
>>>>>>> On Sat, Aug 31, 2024 at 10:17 PM Jens Scheffler
>>>>> <j_scheff...@gmx.de.invalid>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi Ash,
>>>>>>>> 
>>>>>>>> I was thinking hard... was setting the email aside and still have no
>>>>>>>> real _good_ ideas. I am still good with "models" and "sdk".
>>>>>>>> 
>>>>>>>> Actually what we want to define is an "execution interface" to which
>>>>> the
>>>>>>>> structual model as API in Python/or other language gives bindings and
>>>>>>>> helper methods. For the application it is around DAGs - but naming it
>>>>>>>> DAGs is not good because other non-DAG parts as side objects also
>>> need
>>>>>>>> to belong there.
>>>>>>>> 
>>>>>>>> Other terms which came into my mind were "Schema", "System" and
>>> "Plan"
>>>>>>>> but all of there are not as good as the previous "models" or "SDK".
>>>>>>>> 
>>>>>>>> API by the way is too brad and generic and smells like remote. So it
>>>>>>>> should _not_ be "API".
>>>>>>>> 
>>>>>>>> The term "Definitions" is a bit too long in my view.
>>>>>>>> 
>>>>>>>> So... TLDR... this email is not much of help other than saying that
>>> I'd
>>>>>>>> propose to use "airflow.models" or "airflow.sdk". If there are no
>>> other
>>>>>>>> / better ideas coming :-D
>>>>>>>> 
>>>>>>>> Jens
>>>>>>>> 
>>>>>>>> On 30.08.24 19:03, Ash Berlin-Taylor wrote:
>>>>>>>>>> As a side note, I wonder if we should do the user-internal
>>> separation
>>>>>>>> better for DagRun and TaskInstance
>>>>>>>>> Yes, that is a somewhat inevitable side effect of making it be
>>> behind
>>>>>>>> an API, and one I am looking forward to. There are almost just
>>>>> plain-data
>>>>>>>> classes (but not using data classes per se) so we have two different
>>>>>>>> classes — one that is the API representation, and an separate
>>> internal
>>>>> one
>>>>>>>> used by scheduler etc that will have all of the scheduling logic
>>>>> methods.
>>>>>>>>> 
>>>>>>>>> -ash
>>>>>>>>> 
>>>>>>>>>> On 30 Aug 2024, at 17:55, Tzu-ping Chung <t...@astronomer.io.INVALID
>>>> 
>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> On 30 Aug 2024, at 17:48, Ash Berlin-Taylor <a...@apache.org>
>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Where should DAG, TaskGroup, Labels, decorators etc for authoring
>>> be
>>>>>>>> imported from inside the DAG files? Similarly for DagRun,
>>> TaskInstance
>>>>>>>> (these two likely won’t be created directly by users, but just used
>>> for
>>>>>>>> reference docs/type hints)
>>>>>>>>>>> 
>>>>>>>>>> How about airflow.definitions? When discussing assets there’s a
>>>>>>>> question raised on how we should call “DAG files” going forward
>>>>> (because
>>>>>>>> those files now may not contain user-defined DAGs at all).
>>> “Definition
>>>>>>>> files” was raised as a choice, but there’s no existing usage and it
>>>>> might
>>>>>>>> be a bit to catch on. If we put all these things into
>>>>> airflow.definitions,
>>>>>>>> maybe people will start using that term?
>>>>>>>>>> 
>>>>>>>>>> As a side note, I wonder if we should do the user-internal
>>> separation
>>>>>>>> better for DagRun and TaskInstance. We already have that separation
>>> for
>>>>>>>> DAG/DagModel, Dataset/DatasetModel, and more. Maybe we should also
>>> have
>>>>>>>> constructs that users only see, and are converted to “real” objects
>>>>> (i.e.
>>>>>>>> exists in the db) for the scheduler. We already sort of have those in
>>>>>>>> DagRunPydantic and TaskInstancePydantic, we just need to name them
>>>>> better
>>>>>>>> and expose them at the right places.
>>>>>>>>>> 
>>>>>>>>>> TP
>>>>>>>>>> 
>>> ---------------------------------------------------------------------
>>>>>>>>>> To unsubscribe, e-mail: dev-unsubscr...@airflow.apache.org
>>>>>>>>>> For additional commands, e-mail: dev-h...@airflow.apache.org
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>> ---------------------------------------------------------------------
>>>>>>>>> To unsubscribe, e-mail: dev-unsubscr...@airflow.apache.org
>>>>>>>>> For additional commands, e-mail: dev-h...@airflow.apache.org
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> ---------------------------------------------------------------------
>>>>>>>> To unsubscribe, e-mail: dev-unsubscr...@airflow.apache.org
>>>>>>>> For additional commands, e-mail: dev-h...@airflow.apache.org
>>>>>>>> 
>>>>>>>> 
>>>>> 
>>>>> 
>>> 
>>> 
>> 
>> --
>> -Fritz Davenport
>> Senior Data Engineer & CETA Team Lead, Customer Dept @ Astronomer
> 

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscr...@airflow.apache.org
For additional commands, e-mail: dev-h...@airflow.apache.org

Reply via email to