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 2022/02/09 14:04:05 UTC

[GitHub] [airflow] ephraimbuddy opened a new pull request #21462: Refactor airflow db init

ephraimbuddy opened a new pull request #21462:
URL: https://github.com/apache/airflow/pull/21462


   Currently, airflow db init goes through DB upgrade, touching all the migration files
   before getting an up-to-date database.
   SQLAlchemy supports building an up-to-date database by looking through the ORM using metadata.create_all.
   In this PR, the initialization is now done through metadata.create_all, and since we have some FAB models in airflow,
   I think it makes sense to include it in our Base.
   
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/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/main/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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r808073526



##########
File path: tests/utils/test_db.py
##########
@@ -50,23 +50,6 @@ def test_database_schema_and_sqlalchemy_model_are_in_sync(self):
             # ignore indices created by celery
             lambda t: (t[0] == 'remove_index' and t[1].name == 'task_id'),
             lambda t: (t[0] == 'remove_index' and t[1].name == 'taskset_id'),
-            # Ignore all the fab tables
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_register_user'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission_view'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission_view_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_user_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_user'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_view_menu'),
-            # Ignore all the fab indices
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'permission_id'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'name'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'user_id'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'username'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'field_string'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'email'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'permission_view_id'),

Review comment:
       We have to monitor these models since they are now part of airflow




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r808073526



##########
File path: tests/utils/test_db.py
##########
@@ -50,23 +50,6 @@ def test_database_schema_and_sqlalchemy_model_are_in_sync(self):
             # ignore indices created by celery
             lambda t: (t[0] == 'remove_index' and t[1].name == 'task_id'),
             lambda t: (t[0] == 'remove_index' and t[1].name == 'taskset_id'),
-            # Ignore all the fab tables
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_register_user'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission_view'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_permission_view_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_user_role'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_user'),
-            lambda t: (t[0] == 'remove_table' and t[1].name == 'ab_view_menu'),
-            # Ignore all the fab indices
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'permission_id'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'name'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'user_id'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'username'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'field_string'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'email'),
-            lambda t: (t[0] == 'remove_index' and t[1].name == 'permission_view_id'),

Review comment:
       We have to monitor these model since it's now part of airflow




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803157245



##########
File path: airflow/models/taskinstance.py
##########
@@ -386,6 +385,7 @@ class TaskInstance(Base, LoggingMixin):
         Index('ti_pool', pool, state, priority_weight),
         Index('ti_job_id', job_id),
         Index('ti_trigger_id', trigger_id),
+        Index('ti_map_index', dag_id, task_id, run_id, map_index, unique=True),

Review comment:
       Both MySQL and MSSQL required this. Here is the error from MySQL:
   ```sql
   sqlalchemy.exc.OperationalError: (MySQLdb._exceptions.OperationalError) (1822, "Failed to add the foreign key constraint. Missing index for constraint 'task_map_task_instance_fkey' in the referenced table 'task_instance'")
   [SQL: 
   CREATE TABLE task_map (
           dag_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
           task_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
           run_id VARCHAR(250) COLLATE utf8mb3_bin NOT NULL, 
           map_index INTEGER NOT NULL, 
           length INTEGER NOT NULL, 
           `keys` JSON, 
           PRIMARY KEY (dag_id, task_id, run_id, map_index), 
           CONSTRAINT task_map_length_not_negative CHECK (length >= 0), 
           CONSTRAINT task_map_task_instance_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE
   )
   ```
   The unique=True was required by MSSQL:
   ```sql
   sqlalchemy.exc.ProgrammingError: (pyodbc.ProgrammingError) ('42000', "[42000] [Microsoft][ODBC Driver 17 for SQL Server][SQL Server]There are no primary or candidate keys in the referenced table 'task_instance' that match the referencing column list in the foreign key 'task_map_task_instance_fkey'. (1776) (SQLExecDirectW)")
   [SQL: 
   CREATE TABLE task_map (
           dag_id VARCHAR(250) NOT NULL, 
           task_id VARCHAR(250) NOT NULL, 
           run_id VARCHAR(250) NOT NULL, 
           map_index INTEGER NOT NULL, 
           length INTEGER NOT NULL, 
           keys NVARCHAR(max) NULL, 
           PRIMARY KEY (dag_id, task_id, run_id, map_index), 
           CONSTRAINT task_map_length_not_negative CHECK (length >= 0), 
           CONSTRAINT task_map_task_instance_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE
   )
   ```
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] jedcunningham commented on pull request #21462: Refactor airflow db init

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


   Just for some context, this took `db init` on my machine from ~15 seconds to ~13 seconds. I was really hoping it'd make a bigger difference. My unscientific "I watched the log spewing" feeling is that the permissions stuff is the slower part.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r804896684



##########
File path: airflow/models/taskinstance.py
##########
@@ -386,6 +385,7 @@ class TaskInstance(Base, LoggingMixin):
         Index('ti_pool', pool, state, priority_weight),
         Index('ti_job_id', job_id),
         Index('ti_trigger_id', trigger_id),
+        Index('ti_map_index', dag_id, task_id, run_id, map_index, unique=True),

Review comment:
       I'm currently blocked by how to get the dialect name of a database in the ORM and decide whether to add an index or not




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803655807



##########
File path: airflow/models/taskinstance.py
##########
@@ -386,6 +385,7 @@ class TaskInstance(Base, LoggingMixin):
         Index('ti_pool', pool, state, priority_weight),
         Index('ti_job_id', job_id),
         Index('ti_trigger_id', trigger_id),
+        Index('ti_map_index', dag_id, task_id, run_id, map_index, unique=True),

Review comment:
       I was surprised too




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on pull request #21462: Refactor airflow db init

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


   > Not sure what to do on this error from MSSQL:
   > 
   > ```sql
   > sqlalchemy.exc.ProgrammingError: (pyodbc.ProgrammingError) ('42000', "[42000] [Microsoft][ODBC Driver 17 for SQL Server][SQL Server]Introducing FOREIGN KEY constraint 'task_reschedule_dr_fkey' on table 'task_reschedule' may cause cycles or multiple cascade paths. Specify ON DELETE NO ACTION or ON UPDATE NO ACTION, or modify other FOREIGN KEY constraints. (1785) (SQLExecDirectW)")
   > [SQL: 
   > CREATE TABLE task_reschedule (
   >         id INTEGER NOT NULL IDENTITY, 
   >         task_id VARCHAR(250) NOT NULL, 
   >         dag_id VARCHAR(250) NOT NULL, 
   >         run_id VARCHAR(250) NOT NULL, 
   >         map_index INTEGER NOT NULL DEFAULT -1, 
   >         try_number INTEGER NOT NULL, 
   >         start_date DATETIME NOT NULL, 
   >         end_date DATETIME NOT NULL, 
   >         duration INTEGER NOT NULL, 
   >         reschedule_date DATETIME NOT NULL, 
   >         PRIMARY KEY (id), 
   >         CONSTRAINT task_reschedule_ti_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE, 
   >         CONSTRAINT task_reschedule_dr_fkey FOREIGN KEY(dag_id, run_id) REFERENCES dag_run (dag_id, run_id) ON DELETE CASCADE
   > )
   > ```
   
   This might be a real problem...


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on pull request #21462: Refactor airflow db init

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


   Not sure what to do on this error from MSSQL:
   ```sql
   sqlalchemy.exc.ProgrammingError: (pyodbc.ProgrammingError) ('42000', "[42000] [Microsoft][ODBC Driver 17 for SQL Server][SQL Server]Introducing FOREIGN KEY constraint 'task_reschedule_dr_fkey' on table 'task_reschedule' may cause cycles or multiple cascade paths. Specify ON DELETE NO ACTION or ON UPDATE NO ACTION, or modify other FOREIGN KEY constraints. (1785) (SQLExecDirectW)")
   [SQL: 
   CREATE TABLE task_reschedule (
           id INTEGER NOT NULL IDENTITY, 
           task_id VARCHAR(250) NOT NULL, 
           dag_id VARCHAR(250) NOT NULL, 
           run_id VARCHAR(250) NOT NULL, 
           map_index INTEGER NOT NULL DEFAULT -1, 
           try_number INTEGER NOT NULL, 
           start_date DATETIME NOT NULL, 
           end_date DATETIME NOT NULL, 
           duration INTEGER NOT NULL, 
           reschedule_date DATETIME NOT NULL, 
           PRIMARY KEY (id), 
           CONSTRAINT task_reschedule_ti_fkey FOREIGN KEY(dag_id, task_id, run_id, map_index) REFERENCES task_instance (dag_id, task_id, run_id, map_index) ON DELETE CASCADE, 
           CONSTRAINT task_reschedule_dr_fkey FOREIGN KEY(dag_id, run_id) REFERENCES dag_run (dag_id, run_id) ON DELETE CASCADE
   )
   ```
   
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r809744985



##########
File path: airflow/utils/db.py
##########
@@ -663,8 +669,8 @@ def check_migrations(timeout):
     ) as env, settings.engine.connect() as connection:
         env.configure(connection)
         context = env.get_context()
-        source_heads = None
-        db_heads = None
+        source_heads = set(script_.get_heads())
+        db_heads = set(context.get_current_heads())

Review comment:
       ```suggestion
           source_heads = None
           db_heads = None
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk edited a comment on pull request #21462: Create migrations for new DBs from the ORM

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


   > How do we unit test the migrations now? Is that already covered (I have half a memory that you added upgrade and downgrade tests recently?
   
   Not yet merged #21273 . But the "up" migration is always performed (and always have been performed) in all tests as we run  `airflow db upgrade" for all tests we run as the first step.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803860986



##########
File path: airflow/models/taskreschedule.py
##########
@@ -65,7 +66,7 @@ class TaskReschedule(Base):
             [dag_id, run_id],
             ['dag_run.dag_id', 'dag_run.run_id'],
             name='task_reschedule_dr_fkey',
-            ondelete='CASCADE',
+            ondelete='CASCADE' if engine.dialect.name == 'mysql' else 'NO ACTION',

Review comment:
       ```suggestion
               ondelete='CASCADE' if engine.dialect.name != 'mssql' else 'NO ACTION',
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21462: Create migrations for new DBs from the ORM

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






-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy closed pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy closed pull request #21462:
URL: https://github.com/apache/airflow/pull/21462


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on pull request #21462: Refactor airflow db init

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


   The webserver is able to add all the permissions that we need so I'm removing adding of permissions here


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803159456



##########
File path: airflow/models/taskinstance.py
##########
@@ -365,7 +364,7 @@ class TaskInstance(Base, LoggingMixin):
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
 
     # The trigger to resume on if we are in state DEFERRED
-    trigger_id = Column(BigInteger)
+    trigger_id = Column(Integer)

Review comment:
       All the DB doesn't like this and we are also not using BigInteger in the migration file. From mssql:
   ```
   sqlalchemy.exc.ProgrammingError: (pyodbc.ProgrammingError) ('42000', "[42000] [Microsoft][ODBC Driver 17 for SQL Server][SQL Server]Column 'trigger.id' is not the same data type as referencing column 'task_instance.trigger_id' in foreign key 'task_instance_trigger_id_fkey'.
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ashb commented on pull request #21462: Create migrations for new DBs from the ORM

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


   How do we unit test the migrations now? Is that already covered (I have half a memory that you added upgrade and downgrade tests recently?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r809747431



##########
File path: airflow/migrations/versions/c381b21cb7e4_add_session_table_to_db.py
##########
@@ -1,54 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-"""add session table to db
-
-Revision ID: c381b21cb7e4
-Revises: be2bfac3da23
-Create Date: 2022-01-25 13:56:35.069429
-
-"""
-
-import sqlalchemy as sa
-from alembic import op
-
-# revision identifiers, used by Alembic.
-revision = 'c381b21cb7e4'
-down_revision = 'be2bfac3da23'
-branch_labels = None
-depends_on = None
-
-TABLE_NAME = 'session'
-
-
-def upgrade():
-    """Apply add session table to db"""
-    op.create_table(
-        TABLE_NAME,
-        sa.Column('id', sa.Integer()),
-        sa.Column('session_id', sa.String(255)),
-        sa.Column('data', sa.LargeBinary()),
-        sa.Column('expiry', sa.DateTime()),
-        sa.PrimaryKeyConstraint('id'),
-        sa.UniqueConstraint('session_id'),
-    )
-

Review comment:
       Since this can be created externally, I added it in the app's initialization so we don't need the migration again




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803723592



##########
File path: airflow/models/taskinstance.py
##########
@@ -386,6 +385,7 @@ class TaskInstance(Base, LoggingMixin):
         Index('ti_pool', pool, state, priority_weight),
         Index('ti_job_id', job_id),
         Index('ti_trigger_id', trigger_id),
+        Index('ti_map_index', dag_id, task_id, run_id, map_index, unique=True),

Review comment:
       Can we only add this for those two databases? The task instance table is generally pretty huge and it’d be worthwhile to avoid needing this index if possible.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803860986



##########
File path: airflow/models/taskreschedule.py
##########
@@ -65,7 +66,7 @@ class TaskReschedule(Base):
             [dag_id, run_id],
             ['dag_run.dag_id', 'dag_run.run_id'],
             name='task_reschedule_dr_fkey',
-            ondelete='CASCADE',
+            ondelete='CASCADE' if engine.dialect.name == 'mysql' else 'NO ACTION',

Review comment:
       ```suggestion
               ondelete='CASCADE' if engine.dialect.name != 'mysql' else 'NO ACTION',
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803711536



##########
File path: airflow/utils/sqlalchemy.py
##########
@@ -53,7 +55,11 @@ class UtcDateTime(TypeDecorator):
 
     """
 
-    impl = DateTime(timezone=True)
+    impl = (
+        sqla_TIMESTAMP(timezone=True)
+        .with_variant(DATETIME2(precision=6), 'mssql')

Review comment:
       This would mean that `new` DBs won't take `mssql ` version 2000 & 2005




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ashb commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r805852106



##########
File path: airflow/models/taskinstance.py
##########
@@ -2326,6 +2327,35 @@ def filter_for_tis(tis: Iterable[Union["TaskInstance", TaskInstanceKey]]) -> Opt
             ).in_([ti.key.primary for ti in tis])
 
 
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mysql(*arg, **kw):
+    """Add an index for mysql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+        )
+    ).execute_if(dialect='mysql')(*arg, **kw)
+
+
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mssql(*arg, **kw):
+    """Add unique index for mssql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+            unique=True,
+        )
+    ).execute_if(dialect='mssql')(*arg, **kw)

Review comment:
       https://docs.sqlalchemy.org/en/14/core/ddl.html#sqlalchemy.schema.DDLElement.execute_if.params.dialect
   
   `DDL('something').execute_if(dialect=('postgresql', 'mysql'))`




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ashb commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r805927807



##########
File path: airflow/models/taskinstance.py
##########
@@ -2326,6 +2327,35 @@ def filter_for_tis(tis: Iterable[Union["TaskInstance", TaskInstanceKey]]) -> Opt
             ).in_([ti.key.primary for ti in tis])
 
 
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mysql(*arg, **kw):
+    """Add an index for mysql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+        )
+    ).execute_if(dialect='mysql')(*arg, **kw)
+
+
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mssql(*arg, **kw):
+    """Add unique index for mssql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+            unique=True,
+        )
+    ).execute_if(dialect='mssql')(*arg, **kw)

Review comment:
       Isn't this the PK anyway? So it's already unique effectively and zero harm in making it unique AFAICT




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21462: Create migrations for new DBs from the ORM

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


   > How do we unit test the migrations now? Is that already covered (I have half a memory that you added upgrade and downgrade tests recently?
   
   Not yet merged #21273 . But the "up" migration is always performed (and always have been performed) in all tests as we run `airflow upgrade db" for all tests we run as the first step.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803624672



##########
File path: airflow/models/taskinstance.py
##########
@@ -386,6 +385,7 @@ class TaskInstance(Base, LoggingMixin):
         Index('ti_pool', pool, state, priority_weight),
         Index('ti_job_id', job_id),
         Index('ti_trigger_id', trigger_id),
+        Index('ti_map_index', dag_id, task_id, run_id, map_index, unique=True),

Review comment:
       Hmm wait, this is exactly TI’s primary key, why do we need another separate index.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ashb commented on pull request #21462: Create migrations for new DBs from the ORM

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


   How do we unit test the migrations now? Is that already covered (I have half a memory that you added upgrade and downgrade tests recently?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21462: Create migrations for new DBs from the ORM

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


   Though this is migration for empty tables, so not extremely comprehensive.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r805854701



##########
File path: airflow/models/taskinstance.py
##########
@@ -2326,6 +2327,35 @@ def filter_for_tis(tis: Iterable[Union["TaskInstance", TaskInstanceKey]]) -> Opt
             ).in_([ti.key.primary for ti in tis])
 
 
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mysql(*arg, **kw):
+    """Add an index for mysql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+        )
+    ).execute_if(dialect='mysql')(*arg, **kw)
+
+
+@event.listens_for(TaskInstance.__table__, "after_create")
+def add_index_taskinstance_mssql(*arg, **kw):
+    """Add unique index for mssql"""
+    return CreateIndex(
+        Index(
+            'ti_map_index',
+            TaskInstance.__table__.c.dag_id,
+            TaskInstance.__table__.c.task_id,
+            TaskInstance.__table__.c.run_id,
+            TaskInstance.__table__.c.map_index,
+            unique=True,
+        )
+    ).execute_if(dialect='mssql')(*arg, **kw)

Review comment:
       I had to break it into two because `mysql` doesn't require the index to be `unique`. Should I make it unique?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk edited a comment on pull request #21462: Create migrations for new DBs from the ORM

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


   > How do we unit test the migrations now? Is that already covered (I have half a memory that you added upgrade and downgrade tests recently?
   
   Not yet merged #21273 . But the "up" migration is always performed (and always have been performed) in all tests as we run  `airflow db upgrade" for all tests we run as the first step.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on pull request #21462: Refactor airflow db init

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


   > This might be a real problem...
   
   Resolved.
   
   One good thing with going from ORM to migration is the autogeneration of migration files. Once there's a change in a model, we could ask alembic to autogenerate the migration file with the command:
   `alembic revision -m "add mynewcolumn to DagRun" --autogenerate` and it'll generate the file with the changes


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #21462: Refactor airflow db init

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #21462:
URL: https://github.com/apache/airflow/pull/21462#discussion_r803159456



##########
File path: airflow/models/taskinstance.py
##########
@@ -365,7 +364,7 @@ class TaskInstance(Base, LoggingMixin):
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
 
     # The trigger to resume on if we are in state DEFERRED
-    trigger_id = Column(BigInteger)
+    trigger_id = Column(Integer)

Review comment:
       None of the DB likes this and we are also not using BigInteger in the migration file. From mssql:
   ```
   sqlalchemy.exc.ProgrammingError: (pyodbc.ProgrammingError) ('42000', "[42000] [Microsoft][ODBC Driver 17 for SQL Server][SQL Server]Column 'trigger.id' is not the same data type as referencing column 'task_instance.trigger_id' in foreign key 'task_instance_trigger_id_fkey'.
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on pull request #21462: Refactor airflow db init

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


   > Just for some context, this took `db init` on my machine from ~15 seconds to ~13 seconds. I was really hoping it'd make a bigger difference. My unscientific "I watched the log spewing" feeling is that the permissions stuff is the slower part.
   
   Yeah. adding permissions is what takes the most time. My hope is that in the future we could be able to prune the migration files following this change. Also, going from ORM to migration helps in understanding the needs of the different databases.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy closed pull request #21462: Create migrations for new DBs from the ORM

Posted by GitBox <gi...@apache.org>.
ephraimbuddy closed pull request #21462:
URL: https://github.com/apache/airflow/pull/21462


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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