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 2021/12/09 12:21:06 UTC

[GitHub] [airflow] uranusjr opened a new pull request #20165: Log filename template records

uranusjr opened a new pull request #20165:
URL: https://github.com/apache/airflow/pull/20165


   See https://github.com/apache/airflow/issues/19058#issuecomment-947023469 and #19625 for context. This is the main prerequisite to make it possible for us to change the `log_filename_template` config’s default value.
   
   This adds a new table `LogFilename`, and whenever an Airflow command is run (except those explicit set `check_db=False`), the user config value of `log_filename_template` is sync-ed into the table. Each `DagRun` gets a new `log_filename_id` foreign key (populated on creation) that can be used to look up what template they use to render task log filenames. All existing DagRun rows set this value to NULL (for performance reasons), and internally this makes them all use the first row in `LogFilename`, which should be the value in use when a user upgrades to 2.3.
   
   The first commit in from #20163; that one needs to be merged first.


-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/tasklog.py
##########
@@ -0,0 +1,40 @@
+#
+# 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.
+
+from sqlalchemy import Column, Integer, Text
+
+from airflow.models.base import Base
+from airflow.utils import timezone
+from airflow.utils.sqlalchemy import UtcDateTime
+
+
+class LogFilename(Base):
+    """Model to store ``[core] log_filename_template`` config changes.
+
+    This table is automatically populated when Airflow starts up, to store the
+    config's value if it does not match the last row in the table.
+    """
+
+    __tablename__ = "log_filename"
+
+    id = Column(Integer, primary_key=True)
+    template = Column(Text, nullable=False)
+    timestamp = Column(UtcDateTime, nullable=False, default=timezone.utcnow)

Review comment:
       This is just informational, right?
   
   Call it `created_at` maybe? 




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/migrations/versions/f9da662e7089_add_task_log_filename_template_model.py
##########
@@ -0,0 +1,63 @@
+#
+# 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 model for task log filename template.
+
+Revision ID: f9da662e7089
+Revises: 5e3ec427fdd3
+Create Date: 2021-12-09 06:11:21.044940
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# Revision identifiers, used by Alembic.
+revision = "f9da662e7089"
+down_revision = "5e3ec427fdd3"
+branch_labels = None
+depends_on = None
+
+
+def upgrade():
+    """Add model for task log filename template and establish fk on task instance."""
+    op.create_table(
+        "log_filename",
+        sa.Column(sa.Integer, primary_key=True),
+        sa.Column("template", sa.Text, nullable=False),
+        sa.Column("timestamp", sa.UtcDateTime, nullable=False),
+    )
+    with op.batch_alter_table("task_instance") as batch_op:
+        batch_op.add_column(sa.Column("log_filename_id"))

Review comment:
       I switched to use `default` instead. If I read the SQLAlchemy docs correctly, supplying an expression would make it use the expression as an subquery in `INSERT INTO`?




-- 
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 pull request #20165: Log filename template records

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


   Test failures are likely unrelated (one flaky scheduler test failure as usual, the MSSQL job exploded). This should be good to go.


-- 
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 #20165: Log filename template records

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


   > Eh MSSQL does not support `ON DELETE RESTRICT` facepalm
   
   Oh yeah. Unfortunately SQLAlchemy "common" interface starts to break easily when we start using more and more sophisticated DB functionality.


-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/tasklog.py
##########
@@ -0,0 +1,40 @@
+#
+# 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.
+
+from sqlalchemy import Column, Integer, Text
+
+from airflow.models.base import Base
+from airflow.utils import timezone
+from airflow.utils.sqlalchemy import UtcDateTime
+
+
+class LogFilename(Base):
+    """Model to store ``[core] log_filename_template`` config changes.
+
+    This table is automatically populated when Airflow starts up, to store the
+    config's value if it does not match the last row in the table.
+    """
+
+    __tablename__ = "log_filename"
+
+    id = Column(Integer, primary_key=True)
+    template = Column(Text, nullable=False)
+    timestamp = Column(UtcDateTime, nullable=False, default=timezone.utcnow)

Review comment:
       Yeah only informational. The name is borrowed from `XCom`; I like `created_at` more myself as well.




-- 
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 merged pull request #20165: Log filename template records

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


   


-- 
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] github-actions[bot] commented on pull request #20165: Log filename template records

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #20165:
URL: https://github.com/apache/airflow/pull/20165#issuecomment-996652677


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -931,3 +945,13 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
             )
 
         return count
+
+    @provide_session
+    def get_log_filename_template(self, *, session: Session = NEW_SESSION) -> Optional[str]:
+        if self.log_filename_id is None:  # DagRun created before LogFilename introduction.
+            template = session.query(LogFilename.template).order_by(LogFilename.id).limit(1).scalar()
+        else:
+            template = session.query(LogFilename.template).filter_by(id=self.log_filename_id).one_or_none()

Review comment:
       Neither of the branches should ever return None, in fact, because we always ensure there’s a row on startup. I’ll change this to raise an error if None is returned, that makes things more explicit.




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -125,6 +131,12 @@ class DagRun(Base, LoggingMixin):
             mssql_where=text("state='queued'"),
             sqlite_where=text("state='queued'"),
         ),
+        ForeignKeyConstraint(
+            [log_filename_id],
+            ["log_filename.id"],
+            name="task_instance_log_filename_id_fkey",
+            ondelete="RESTRICT",
+        ),

Review comment:
       `RESTRICT` means a `LogFilename` row cannot be deleted unless no `DagRun` is referencing it.




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -931,3 +945,13 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
             )
 
         return count
+
+    @provide_session
+    def get_log_filename_template(self, *, session: Session = NEW_SESSION) -> Optional[str]:
+        if self.log_filename_id is None:  # DagRun created before LogFilename introduction.
+            template = session.query(LogFilename.template).order_by(LogFilename.id).limit(1).scalar()
+        else:
+            template = session.query(LogFilename.template).filter_by(id=self.log_filename_id).one_or_none()

Review comment:
       Oh get only works on objects, but we are asking for a single column. TIL.




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/migrations/versions/f9da662e7089_add_task_log_filename_template_model.py
##########
@@ -0,0 +1,63 @@
+#
+# 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 model for task log filename template.
+
+Revision ID: f9da662e7089
+Revises: 5e3ec427fdd3
+Create Date: 2021-12-09 06:11:21.044940
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# Revision identifiers, used by Alembic.
+revision = "f9da662e7089"
+down_revision = "5e3ec427fdd3"
+branch_labels = None
+depends_on = None
+
+
+def upgrade():
+    """Add model for task log filename template and establish fk on task instance."""
+    op.create_table(
+        "log_filename",
+        sa.Column(sa.Integer, primary_key=True),
+        sa.Column("template", sa.Text, nullable=False),
+        sa.Column("timestamp", sa.UtcDateTime, nullable=False),
+    )
+    with op.batch_alter_table("task_instance") as batch_op:
+        batch_op.add_column(sa.Column("log_filename_id"))

Review comment:
       I think the `server_default` (or equivalent) needs to go here for it to have any effect: https://docs.sqlalchemy.org/en/13/core/defaults.html#server-defaults
   
   > A variant on the SQL expression default is the Column.server_default, which gets placed in the CREATE TABLE statement during a Table.create() operation




-- 
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 pull request #20165: Log filename template records

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


   Oh because I just merged a PR with model 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] kaxil commented on pull request #20165: Log filename template records

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


   Can you rebase on main and fix conflicts too plz


-- 
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 pull request #20165: Log filename template records

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


   Changed to `ON DELETE NO ACTION` instead. This should work well enough.


-- 
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 pull request #20165: Log filename template records

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


   Eh MSSQL does not support `ON DELETE RESTRICT` 🤦 


-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -931,3 +945,13 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
             )
 
         return count
+
+    @provide_session
+    def get_log_filename_template(self, *, session: Session = NEW_SESSION) -> Optional[str]:
+        if self.log_filename_id is None:  # DagRun created before LogFilename introduction.
+            template = session.query(LogFilename.template).order_by(LogFilename.id).limit(1).scalar()
+        else:
+            template = session.query(LogFilename.template).filter_by(id=self.log_filename_id).one_or_none()

Review comment:
       ```suggestion
               template = session.query(LogFilename.template).get(self.log_filename_id)
   ```
   
   This shouldn't ever return None because of the RESTRICT FK.




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -931,3 +945,13 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
             )
 
         return count
+
+    @provide_session
+    def get_log_filename_template(self, *, session: Session = NEW_SESSION) -> Optional[str]:
+        if self.log_filename_id is None:  # DagRun created before LogFilename introduction.
+            template = session.query(LogFilename.template).order_by(LogFilename.id).limit(1).scalar()
+        else:
+            template = session.query(LogFilename.template).filter_by(id=self.log_filename_id).one_or_none()
+        if template is not None:
+            return template
+        return airflow_conf.get("logging", "LOG_FILENAME_TEMPLATE")

Review comment:
       I’m not sure if setting up a relationship (for caching) is worthwile. Maybe?




-- 
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 #20165: Log filename template records

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



##########
File path: airflow/models/dagrun.py
##########
@@ -931,3 +945,13 @@ def schedule_tis(self, schedulable_tis: Iterable[TI], session: Session = NEW_SES
             )
 
         return count
+
+    @provide_session
+    def get_log_filename_template(self, *, session: Session = NEW_SESSION) -> Optional[str]:
+        if self.log_filename_id is None:  # DagRun created before LogFilename introduction.
+            template = session.query(LogFilename.template).order_by(LogFilename.id).limit(1).scalar()
+        else:
+            template = session.query(LogFilename.template).filter_by(id=self.log_filename_id).one_or_none()

Review comment:
       Turns out `get()` cannot be used here: `get() can only be used against a single mapped class`.




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