As I mentioned multiple times in similar discussions We have a huge problem with unicode in dag_id. Namely MySQL limit on indexes. We would have to shorten the Id significantly in the database to workaround MySQL limits for index size.
We can have a wishful thinking that we can change dag_id to unicode but until someone solves the problem - this is just this - wishful thinking. If someone has a proposal how to do it without breaking compatibility or enormously complicating mysql case (or if we drop mysql proposal) - I would also be for what Daniel said. But so far I have not seen any. So in the absence of a viable way to add unicode to dag_id (which currently IMHO is not an option) my vote goes to 2. We can also drop MySQL support :D J. On Thu, Jan 12, 2023 at 9:56 AM Ash Berlin-Taylor <[email protected]> wrote: > +1 to what Daniel said > > On 12 January 2023 08:32:29 GMT, Daniel Standish > <[email protected]> wrote: >> >> 1 appears to have potential fix: >> https://github.com/apache/airflow/issues/21127#issuecomment-1030673862 >> 2. seems to fail due to our own ascii enforcement... what if we remove >> that? >> 3. does not appear to be unicode-related or dag_id-related but a feature >> request for user-friendly mapped task aliases... >> >> not saying we should not add a "name" of some kind... but ... does not >> yet seem clear we can't just enable unicode... i know others have given >> this much more thought than I and maybe they can chime in with other >> concerns we may have encountered as this idea has bounced around >> >> On Thu, Jan 12, 2023 at 12:01 AM Abdul Hadi Shakir <[email protected]> >> wrote: >> >>> Directly using non-ascii characters (unicodes included) in *dag_id* breaks >>> couples of functionalities. See issues: >>> >>> - Fail to download task log if there are Chinese characters in dag_id >>> #21127 <https://github.com/apache/airflow/issues/21127> >>> - Airflow scheduler with statsd enabled crashes when dag_id contains >>> unexpected characters #18010 >>> <https://github.com/apache/airflow/issues/18010> >>> - Names for expanded tasks #23020 >>> <https://github.com/apache/airflow/issues/23020> >>> >>> *Abdul Hadi Shakir* >>> >>> >>> On Thu, Jan 12, 2023 at 1:19 PM Daniel Standish >>> <[email protected]> wrote: >>> >>>> Hi, >>>> >>>> Is it not possible to just have unicode dag_id with no distinct >>>> "name"? If you explored this route and encountered problems which caused >>>> you to abandon, can you share what were the problems? >>>> >>>> I think having just one ID for a dag is a nice thing, if we can keep it. >>>> >>>> On Wed, Jan 11, 2023 at 11:43 PM Abdul Hadi Shakir < >>>> [email protected]> wrote: >>>> >>>>> Hi team, >>>>> >>>>> While discussing the approach for >>>>> https://github.com/apache/airflow/issues/22073 (adding support for >>>>> national characters in DAG display name) - two approaches came out. Need >>>>> votes to finalise on one of the two: >>>>> >>>>> 1. [Vote *+1*] Using *name* as the only parameter; and then >>>>> generating a unique *dag_id* from it using *slugify*. This makes >>>>> the interface simpler; but it makes *dag_id* unknown from the >>>>> users. Ongoing PR for this: >>>>> https://github.com/apache/airflow/pull/28183 >>>>> 2. [Vote -*1*] To use *display_name* along with *dag_id* as DAG >>>>> params. While this is a simpler solution on the backend - it needs >>>>> lots of >>>>> work on the frontend for a consistent experience. Ongoing PR for this: >>>>> https://github.com/apache/airflow/pull/27145 >>>>> >>>>> Cheers, >>>>> *Abdul Hadi Shakir* >>>>> >>>>
