You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/17 20:43:03 UTC

[GitHub] [airflow] ashb opened a new pull request #11621: Make DagRunType inherit from `str` too for easier use.

ashb opened a new pull request #11621:
URL: https://github.com/apache/airflow/pull/11621


   This approach is documented in https://docs.python.org/3.6/library/enum.html#others:
   
   ```
   While IntEnum is part of the enum module, it would be very simple to
   implement independently:
   
   class IntEnum(int, Enum):
       pass
   ```
   
   We just extend this to a str -- this means the SQLAlchemy has no trouble
   putting these in to queries, and `"scheduled" == DagRunType.SCHEDULED`
   is true.
   
   This change makes it simpler to use `dagrun.run_type`.
   
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749738694


   It does seem like db driver specific, not able to reproduce with postgres, strace output:
   
   ```
   sendto(3, "Q\0\0\0\236SELECT dag_run.dag_id AS dag_run_dag_id, max(dag_run.execution_date) AS max_1 \nFROM dag_run \nWHERE dag_run.run_type = 'scheduled' GROUP BY dag_run.dag_id\0", 159, MSG_NOSIGNAL, NULL, 0) = 159
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp edited a comment on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749426713


   @ashb did you test this with mysql as well? I am trying figure out whether I am missing something in my mysql config. When the enum type used in sqlalchemy query, it's being resolved into `'DagRunType.SCHEDULED'` not `'scheduled'`. I am able to reproduce this in breeze with the following code:
   
   ```
   from sqlalchemy import Boolean, Column, ForeignKey, Index, Integer, String, Text, func, or_
   from airflow.models.dagrun import DagRun
   from airflow.utils.types import DagRunType
   from airflow.utils.session import create_session
   
   with create_session() as session:
       q1 = session.query(
               DagRun.dag_id, func.max_(DagRun.execution_date),
       ).filter(
           DagRun.run_type == DagRunType.SCHEDULED,
       ).group_by(DagRun.dag_id)
       q.all()
   ```
   
   Running this script with strace shows that sqlalchemy sent `'DagRunType.SCHEDULED'` to the mysql server instead of `'scheduled'`. Here is the output from strace (`strace -v -s 1024 -e trace=sendto python a.py`):
   
   ```
   sendto(3, "\314\0\0\0\3SELECT dag_run.dag_id AS dag_run_dag_id, max(dag_run.execution_date) AS max_1 \nFROM dag_run \nWHERE dag_run.run_type = 'DagRunType.SCHEDULED' GROUP BY dag_run.dag_id", 208, 0, NULL, 0) = 208
   ```
   
   Using `DagRunType.SCHEDULED.value` in the query fixes the error.
   
   tested with commit: `872350bac5bebea09bd52d50734a3b7517af712c`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp edited a comment on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749771427


   I have isolated this to the `execute` function in `MySQLdb/cursors.py`, it calls `db.literal` on query args, which evaluates enum value `DagRunType.SCHEDULED` to `'DagRunType.SCHEDULED'`. So looks like we can't guarantee consistent enum type handling behaviors between db drivers.
   
   see: https://github.com/PyMySQL/mysqlclient/blob/ca630c01fb39f252a4c91a525e440beea4ac4447/MySQLdb/connections.py#L288
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749715219


   yeah, I was thinking maybe it has something to do with mysql sqlalchemy engine config, which changed how it binds query params for enum types. Let me test this with postgres in breeze to see if it's behaving the same.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp edited a comment on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749426713


   @ashb did you test this with mysql as well? I am trying figure out whether I am missing something in my mysql config. When the enum type used in sqlalchemy query, it's being resolved into `'DagRunType.SCHEDULED'` not `'scheduled'`. I am able to reproduce this in breeze with the following code:
   
   ```
   from sqlalchemy import Boolean, Column, ForeignKey, Index, Integer, String, Text, func, or_
   from airflow.models.dagrun import DagRun
   from airflow.utils.types import DagRunType
   from airflow.utils.session import create_session
   
   with create_session() as session:
       q1 = session.query(
               DagRun.dag_id, func.max_(DagRun.execution_date),
       ).filter(
           DagRun.run_type == DagRunType.SCHEDULED,
       ).group_by(DagRun.dag_id)
       q.all()
   ```
   
   Running this script with strace shows that sqlalchemy sent `'DagRunType.SCHEDULED'` to the mysql server instead of `'scheduled'`. Here is the output from strace (`strace -v -s 1024 -e trace=sendto python a.py`):
   
   ```
   sendto(3, "\314\0\0\0\3SELECT dag_run.dag_id AS dag_run_dag_id, max(dag_run.execution_date) AS max_1 \nFROM dag_run \nWHERE dag_run.run_type = 'DagRunType.SCHEDULED' GROUP BY dag_run.dag_id", 208, 0, NULL, 0) = 208
   ```
   
   Using `DagRunType.SCHEDULED.value` in the query fixes the error.
   
   tested with commit: 872350bac5bebea09bd52d50734a3b7517af712c


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp edited a comment on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749426713


   @ashb did you test this with mysql as well? I am trying figure out whether I am missing something in my mysql config. When the enum type used in sqlalchemy query, it's being resolved into `'DagRunType.SCHEDULED'` not `'scheduled'`. I am able to reproduce this in breeze with the following code:
   
   ```
   from sqlalchemy import Boolean, Column, ForeignKey, Index, Integer, String, Text, func, or_
   from airflow.models.dagrun import DagRun
   from airflow.utils.types import DagRunType
   from airflow.utils.session import create_session
   
   with create_session() as session:
       q1 = session.query(
               DagRun.dag_id, func.max_(DagRun.execution_date),
       ).filter(
           DagRun.run_type == DagRunType.SCHEDULED,
       ).group_by(DagRun.dag_id)
       q.all()
   ```
   
   Running this script with strace shows that sqlalchemy sent `'DagRunType.SCHEDULED'` to the mysql server instead of `'scheduled'`. Here is the output from strace (`strace -v -s 1024 -e trace=sendto python a.py`):
   
   ```
   sendto(3, "\314\0\0\0\3SELECT dag_run.dag_id AS dag_run_dag_id, max(dag_run.execution_date) AS max_1 \nFROM dag_run \nWHERE dag_run.run_type = 'DagRunType.SCHEDULED' GROUP BY dag_run.dag_id", 208, 0, NULL, 0) = 208
   ```
   
   Using `DagRunType.SCHEDULED.value` in the query fixes the error.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb merged pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
ashb merged pull request #11621:
URL: https://github.com/apache/airflow/pull/11621


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-711097845


   Nice!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749426713


   @ashb did you test this with mysql as well? I am trying figure out whether I am missing something in my mysql config. When the enum type used in sqlalchemy query, it's being resolved into `'DagRunType.SCHEDULED'` not `'scheduled'`. I am able to reproduce this in breeze with the following code:
   
   ```
   from sqlalchemy import Boolean, Column, ForeignKey, Index, Integer, String, Text, func, or_
   from airflow.models.dagrun import DagRun
   from airflow.utils.types import DagRunType
   from airflow.utils.session import create_session
   
   with create_session() as session:
       q1 = session.query(
               DagRun.dag_id, func.max_(DagRun.execution_date),
       ).filter(
           DagRun.run_type == DagRunType.SCHEDULED,
       ).group_by(DagRun.dag_id)
       q.all()
   ```
   
   Running this script with strace shows that sqlalchemy sent `'DagRunType.SCHEDULED'` to the mysql server instead of `'scheduled'`. Here is the output from strace (`strace -v -s 1024 -e trace=sendto python a.py`):
   
   ```
   sendto(3, "\314\0\0\0\3SELECT dag_run.dag_id AS dag_run_dag_id, max(dag_run.execution_date) AS max_1 \nFROM dag_run \nWHERE dag_run.run_type = 'DagRunType.SCHEDULED' GROUP BY dag_run.dag_id", 208, 0, NULL, 0) = 208
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp edited a comment on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749771427


   I have isolated this to the `execute` function in `MySQLdb/cursors.py`, it calls `db.literal` on query args, which evaluates enum value `DagRunType.SCHEDULED` to `'DagRunType.SCHEDULED'`. So looks like we can't guarantee consistent enum type handling behaviors between db drivers.
   
   relavent code: https://github.com/PyMySQL/mysqlclient/blob/ca630c01fb39f252a4c91a525e440beea4ac4447/MySQLdb/connections.py#L288
   https://github.com/PyMySQL/mysqlclient/blob/ca630c01fb39f252a4c91a525e440beea4ac4447/MySQLdb/converters.py#L106
   https://github.com/PyMySQL/mysqlclient/blob/master/MySQLdb/_mysql.c#L1027
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] houqp commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749771427


   I have isolated this to the `execute` function in `MySQLdb/cursors.py`, it calls `db.literal` on query args, which evaluates enum value `DagRunType.SCHEDULED` to `'DagRunType.SCHEDULED'`. So looks like we can't guarantee consistent enum type handling behaviors between db drivers.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #11621: Make DagRunType inherit from `str` too for easier use.

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #11621:
URL: https://github.com/apache/airflow/pull/11621#issuecomment-749440842


   Yes, we tested this with mysql - it's not db specific as python translates this before it it gets to a db


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org