[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-23 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r338310718
 
 

 ##
 File path: docs/howto/enable-dag-serialization.rst
 ##
 @@ -0,0 +1,109 @@
+ .. 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.
+
+
+
+
+Enable DAG Serialization
+
+
+Add the following settings in ``airflow.cfg``:
+
+.. code-block:: ini
+
+[core]
+store_serialized_dags = True
+min_serialized_dag_update_interval = 30
+
+*   ``store_serialized_dags``: This flag decides whether to serialises DAGs 
and persist them in DB.
+If set to True, Webserver reads from DB instead of parsing DAG files
+*   ``min_serialized_dag_update_interval``: This flag sets the minimum 
interval (in seconds) after which
+the serialized DAG in DB should be updated. This helps in reducing 
database write rate.
+
+If you are updating Airflow from <1.10.6, please do not forget to run 
``airflow db upgrade``.
+
+
+How it works
+
+
+In order to make Airflow Webserver stateless (almost!), Airflow >=1.10.6 
supports
+DAG Serialization and DB Persistence.
+
+.. image:: ../img/dag_serialization.png
+
+As shown in the image above in Vanilla Airflow, the Webserver and the 
Scheduler both
+needs access to the DAG files. Both the scheduler and webserver parses the DAG 
files.
+
+With **DAG Serialization** we aim to decouple the webserver from DAG parsing
+which would make the Webserver very light-weight.
+
+As shown in the image above, when using the dag_serilization feature,
+the Scheduler parses the DAG files, serializes them in JSON format and saves 
them in the Metadata DB.
+
+The Webserver now instead of having to parse the DAG file again, reads the
+serialized DAGs in JSON, de-serializes them and create the DagBag and uses it
+to show in the UI.
+
+One of the key features that is implemented as the part of DAG Serialization 
is that
+instead of loading an entire DagBag when the WebServer starts we only load 
each DAG on demand from the
+Serialized Dag table. This helps reduce Webserver startup time and memory. The 
reduction is notable
+when you have large number of DAGs.
+
+Below is the screenshot of the ``serialized_dag`` table in Metadata DB:
+
+.. image:: ../img/serialized_dag_table.png
+
+Limitations
+---
+The Webserver will still need access to DAG files in the following cases,
+which is why we said "almost" stateless.
+
+*   **Rendered Template** tab will still have to parse Python file as it needs 
all the details like
+the execution date and even the data passed by the upstream task using 
Xcom.
+*   **Code View** will read the DAG File & show it using Pygments.
+However, it does not need to Parse the Python file so it is still a small 
operation.
+*   :doc:`Extra Operator Links ` would not work out of
+the box. They need to be defined in Airflow Plugin.
+
+**Existing Airflow Operators**:
+To make extra operator links work with existing operators like BigQuery, 
copy all
+the classes that are defined in ``operator_extra_links`` property.
 
 Review comment:
   Should we create a pre-existing plugin and register the existing links there 
so that users can re-use them easily?
   
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-16 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r335609229
 
 

 ##
 File path: docs/howto/enable-dag-serialization.rst
 ##
 @@ -0,0 +1,109 @@
+ .. 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.
+
+
+
+
+Enable DAG Serialization
+
+
+Add the following settings in ``airflow.cfg``:
+
+.. code-block:: ini
+
+[core]
+store_serialized_dags = True
+min_serialized_dag_update_interval = 30
+
+*   ``store_serialized_dags``: This flag decides whether to serialises DAGs 
and persist them in DB.
+If set to True, Webserver reads from DB instead of parsing DAG files
+*   ``min_serialized_dag_update_interval``: This flag sets the minimum 
interval (in seconds) after which
+the serialized DAG in DB should be updated. This helps in reducing 
database write rate.
+
+If you are updating Airflow from <1.10.6, please do not forget to run 
``airflow db upgrade``.
+
+
+How it works
+
+
+In order to make Airflow Webserver stateless (almost!), Airflow >=1.10.6 
supports
+DAG Serialization and DB Persistence.
+
+.. image:: ../img/dag_serialization.png
+
+As shown in the image above in Vanilla Airflow, the Webserver and the 
Scheduler both
+needs access to the DAG files. Both the scheduler and webserver parses the DAG 
files.
+
+With **DAG Serialization** we aim to decouple the webserver from DAG parsing
+which would make the Webserver very light-weight.
+
+As shown in the image above, when using the dag_serilization feature,
+the Scheduler parses the DAG files, serializes them in JSON format and saves 
them in the Metadata DB.
+
+The Webserver now instead of having to parse the DAG file again, reads the
+serialized DAGs in JSON, de-serializes them and create the DagBag and uses it
+to show in the UI.
+
+One of the key features that is implemented as the part of DAG Serialization 
is that
+instead of loading an entire DagBag when the WebServer starts we only load 
each DAG on demand from the
+Serialized Dag table. This helps reduce Webserver startup time and memory. The 
reduction is notable
+when you have large number of DAGs.
+
+Below is the screenshot of the ``serialized_dag`` table in Metadata DB:
+
+.. image:: ../img/serialized_dag_table.png
+
+Limitations
+---
+The Webserver will still need access to DAG files in the following cases,
+which is why we said "almost" stateless.
+
+*   **Rendered Template** tab will still have to parse Python file as it needs 
all the details like
+the execution date and even the data passed by the upstream task using 
Xcom.
+*   **Code View** will read the DAG File & show it using Pygments.
+However, it does not need to Parse the Python file so it is still a small 
operation.
+*   :doc:`Extra Operator Links ` would not work out of
+the box. They need to be defined in Airflow Plugin.
+
+**Existing Airflow Operators**:
+To make extra operator links work with existing operators like BigQuery, 
copy all
+the classes that are defined in ``operator_extra_links`` property.
 
 Review comment:
   Can you clarify a bit about the "bundled" GCP links? Are you talking about 
the operator extra links registered as a `@property` in the operator like:
   
https://github.com/apache/airflow/blob/master/airflow/gcp/operators/bigquery.py#L460-L470
   ?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-15 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r335284375
 
 

 ##
 File path: docs/howto/enable-dag-serialization.rst
 ##
 @@ -0,0 +1,109 @@
+ .. 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.
+
+
+
+
+Enable DAG Serialization
+
+
+Add the following settings in ``airflow.cfg``:
+
+.. code-block:: ini
+
+[core]
+store_serialized_dags = True
+min_serialized_dag_update_interval = 30
+
+*   ``store_serialized_dags``: This flag decides whether to serialises DAGs 
and persist them in DB.
+If set to True, Webserver reads from DB instead of parsing DAG files
+*   ``min_serialized_dag_update_interval``: This flag sets the minimum 
interval (in seconds) after which
+the serialized DAG in DB should be updated. This helps in reducing 
database write rate.
+
+If you are updating Airflow from <1.10.6, please do not forget to run 
``airflow db upgrade``.
+
+
+How it works
+
+
+In order to make Airflow Webserver stateless (almost!), Airflow >=1.10.6 
supports
+DAG Serialization and DB Persistence.
+
+.. image:: ../img/dag_serialization.png
+
+As shown in the image above in Vanilla Airflow, the Webserver and the 
Scheduler both
+needs access to the DAG files. Both the scheduler and webserver parses the DAG 
files.
+
+With **DAG Serialization** we aim to decouple the webserver from DAG parsing
+which would make the Webserver very light-weight.
+
+As shown in the image above, when using the dag_serilization feature,
+the Scheduler parses the DAG files, serializes them in JSON format and saves 
them in the Metadata DB.
+
+The Webserver now instead of having to parse the DAG file again, reads the
+serialized DAGs in JSON, de-serializes them and create the DagBag and uses it
+to show in the UI.
+
+One of the key features that is implemented as the part of DAG Serialization 
is that
+instead of loading an entire DagBag when the WebServer starts we only load 
each DAG on demand from the
+Serialized Dag table. This helps reduce Webserver startup time and memory. The 
reduction is notable
+when you have large number of DAGs.
+
+Below is the screenshot of the ``serialized_dag`` table in Metadata DB:
+
+.. image:: ../img/serialized_dag_table.png
+
+Limitations
+---
+The Webserver will still need access to DAG files in the following cases,
+which is why we said "almost" stateless.
+
+*   **Rendered Template** tab will still have to parse Python file as it needs 
all the details like
+the execution date and even the data passed by the upstream task using 
Xcom.
+*   **Code View** will read the DAG File & show it using Pygments.
+However, it does not need to Parse the Python file so it is still a small 
operation.
+*   :doc:`Extra Operator Links ` would not work out of
+the box. They need to be defined in Airflow Plugin.
+
+**Existing Airflow Operators**:
+To make extra operator links work with existing operators like BigQuery, 
copy all
+the classes that are defined in ``operator_extra_links`` property.
 
 Review comment:
   Does it mean `operator_extra_links` can only be registered through plugin in 
the future? 


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-15 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r335283942
 
 

 ##
 File path: docs/howto/enable-dag-serialization.rst
 ##
 @@ -0,0 +1,109 @@
+ .. 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.
+
+
+
+
+Enable DAG Serialization
+
+
+Add the following settings in ``airflow.cfg``:
+
+.. code-block:: ini
+
+[core]
+store_serialized_dags = True
+min_serialized_dag_update_interval = 30
+
+*   ``store_serialized_dags``: This flag decides whether to serialises DAGs 
and persist them in DB.
+If set to True, Webserver reads from DB instead of parsing DAG files
+*   ``min_serialized_dag_update_interval``: This flag sets the minimum 
interval (in seconds) after which
+the serialized DAG in DB should be updated. This helps in reducing 
database write rate.
+
+If you are updating Airflow from <1.10.6, please do not forget to run 
``airflow db upgrade``.
+
+
+How it works
+
+
+In order to make Airflow Webserver stateless (almost!), Airflow >=1.10.6 
supports
+DAG Serialization and DB Persistence.
+
+.. image:: ../img/dag_serialization.png
+
+As shown in the image above in Vanilla Airflow, the Webserver and the 
Scheduler both
+needs access to the DAG files. Both the scheduler and webserver parses the DAG 
files.
+
+With **DAG Serialization** we aim to decouple the webserver from DAG parsing
+which would make the Webserver very light-weight.
+
+As shown in the image above, when using the dag_serilization feature,
+the Scheduler parses the DAG files, serializes them in JSON format and saves 
them in the Metadata DB.
+
+The Webserver now instead of having to parse the DAG file again, reads the
+serialized DAGs in JSON, de-serializes them and create the DagBag and uses it
+to show in the UI.
+
+One of the key features that is implemented as the part of DAG Serialization 
is that
+instead of loading an entire DagBag when the WebServer starts we only load 
each DAG on demand from the
+Serialized Dag table. This helps reduce Webserver startup time and memory. The 
reduction is notable
+when you have large number of DAGs.
+
+Below is the screenshot of the ``serialized_dag`` table in Metadata DB:
+
+.. image:: ../img/serialized_dag_table.png
+
+Limitations
+---
+The Webserver will still need access to DAG files in the following cases,
+which is why we said "almost" stateless.
+
+*   **Rendered Template** tab will still have to parse Python file as it needs 
all the details like
+the execution date and even the data passed by the upstream task using 
Xcom.
+*   **Code View** will read the DAG File & show it using Pygments.
+However, it does not need to Parse the Python file so it is still a small 
operation.
+*   :doc:`Extra Operator Links ` would not work out of
+the box. They need to be defined in Airflow Plugin.
+
+**Existing Airflow Operators**:
 
 Review comment:
   I think @kaxil is referring to existing operators with `operator extra links`


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-15 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r335280952
 
 

 ##
 File path: airflow/models/serialized_dag.py
 ##
 @@ -0,0 +1,214 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+
+"""Serialzed DAG table in database."""
+
+import hashlib
+from datetime import timedelta
+from typing import TYPE_CHECKING, Any, Dict, List, Optional
+
+# from sqlalchemy import Column, Index, Integer, String, Text, JSON, and_, exc
+from sqlalchemy import JSON, Column, Index, Integer, String, and_
+from sqlalchemy.sql import exists
+
+from airflow.models.base import ID_LEN, Base
+from airflow.utils import db, timezone
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.sqlalchemy import UtcDateTime
+
+if TYPE_CHECKING:
+from airflow.models import DAG  # noqa: F401; # pylint: 
disable=cyclic-import
+from airflow.serialization import SerializedDAG  # noqa: F401
+
+
+log = LoggingMixin().log
+
+
+class SerializedDagModel(Base):
+"""A table for serialized DAGs.
+
+serialized_dag table is a snapshot of DAG files synchronized by scheduler.
+This feature is controlled by:
+
+* ``[core] store_serialized_dags = True``: enable this feature
+* ``[core] min_serialized_dag_update_interval = 30`` (s):
+  serialized DAGs are updated in DB when a file gets processed by 
scheduler,
+  to reduce DB write rate, there is a minimal interval of updating 
serialized DAGs.
+* ``[scheduler] dag_dir_list_interval = 300`` (s):
+  interval of deleting serialized DAGs in DB when the files are deleted, 
suggest
+  to use a smaller interval such as 60
+
+It is used by webserver to load dagbags when 
``store_serialized_dags=True``.
+Because reading from database is lightweight compared to importing from 
files,
+it solves the webserver scalability issue.
+"""
+__tablename__ = 'serialized_dag'
+
+dag_id = Column(String(ID_LEN), primary_key=True)
+fileloc = Column(String(2000), nullable=False)
+# The max length of fileloc exceeds the limit of indexing.
+fileloc_hash = Column(Integer, nullable=False)
+data = Column(JSON, nullable=False)
+last_updated = Column(UtcDateTime, nullable=False)
+
+__table_args__ = (
+Index('idx_fileloc_hash', fileloc_hash, unique=False),
+)
+
+def __init__(self, dag: 'DAG'):
+from airflow.serialization import SerializedDAG  # noqa # pylint: 
disable=redefined-outer-name
+
+self.dag_id = dag.dag_id
+self.fileloc = dag.full_filepath
+self.fileloc_hash = self.dag_fileloc_hash(self.fileloc)
+self.data = SerializedDAG.to_dict(dag)
+self.last_updated = timezone.utcnow()
+
+@staticmethod
+def dag_fileloc_hash(full_filepath: str) -> int:
+Hashing file location for indexing.
+
+:param full_filepath: full filepath of DAG file
+:return: hashed full_filepath
+"""
+# hashing is needed because the length of fileloc is 2000 as an 
Airflow convention,
+# which is over the limit of indexing. If we can reduce the length of 
fileloc, then
+# hashing is not needed.
+return int.from_bytes(
+hashlib.sha1(full_filepath.encode('utf-8')).digest()[-2:], 
byteorder='big', signed=False)
+
+@classmethod
+@db.provide_session
+def write_dag(cls, dag: 'DAG', min_update_interval: Optional[int] = None, 
session=None):
+"""Serializes a DAG and writes it into database.
+
+:param dag: a DAG to be written into database
+:param min_update_interval: minimal interval in seconds to update 
serialized DAG
+:param session: ORM Session
+"""
+log.debug("Writing DAG: %s to the DB", dag)
+# Checks if (Current Time - Time when the DAG was written to DB) < 
min_update_interval
+# If Yes, does nothing
+# If No or the DAG does not exists, updates / writes Serialized DAG to 
DB
+if min_update_interval is not None:
+if session.query(exists().where(
+and_(cls.dag_id == 

[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-10-15 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r335280049
 
 

 ##
 File path: airflow/models/serialized_dag.py
 ##
 @@ -0,0 +1,214 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+
+"""Serialzed DAG table in database."""
+
+import hashlib
+from datetime import timedelta
+from typing import TYPE_CHECKING, Any, Dict, List, Optional
+
+# from sqlalchemy import Column, Index, Integer, String, Text, JSON, and_, exc
+from sqlalchemy import JSON, Column, Index, Integer, String, and_
+from sqlalchemy.sql import exists
+
+from airflow.models.base import ID_LEN, Base
+from airflow.utils import db, timezone
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.sqlalchemy import UtcDateTime
+
+if TYPE_CHECKING:
+from airflow.models import DAG  # noqa: F401; # pylint: 
disable=cyclic-import
+from airflow.serialization import SerializedDAG  # noqa: F401
+
+
+log = LoggingMixin().log
+
+
+class SerializedDagModel(Base):
+"""A table for serialized DAGs.
+
+serialized_dag table is a snapshot of DAG files synchronized by scheduler.
+This feature is controlled by:
+
+* ``[core] store_serialized_dags = True``: enable this feature
+* ``[core] min_serialized_dag_update_interval = 30`` (s):
+  serialized DAGs are updated in DB when a file gets processed by 
scheduler,
+  to reduce DB write rate, there is a minimal interval of updating 
serialized DAGs.
+* ``[scheduler] dag_dir_list_interval = 300`` (s):
+  interval of deleting serialized DAGs in DB when the files are deleted, 
suggest
+  to use a smaller interval such as 60
+
+It is used by webserver to load dagbags when 
``store_serialized_dags=True``.
+Because reading from database is lightweight compared to importing from 
files,
+it solves the webserver scalability issue.
+"""
+__tablename__ = 'serialized_dag'
+
+dag_id = Column(String(ID_LEN), primary_key=True)
+fileloc = Column(String(2000), nullable=False)
+# The max length of fileloc exceeds the limit of indexing.
+fileloc_hash = Column(Integer, nullable=False)
+data = Column(JSON, nullable=False)
+last_updated = Column(UtcDateTime, nullable=False)
+
+__table_args__ = (
+Index('idx_fileloc_hash', fileloc_hash, unique=False),
+)
+
+def __init__(self, dag: 'DAG'):
+from airflow.serialization import SerializedDAG  # noqa # pylint: 
disable=redefined-outer-name
+
+self.dag_id = dag.dag_id
+self.fileloc = dag.full_filepath
+self.fileloc_hash = self.dag_fileloc_hash(self.fileloc)
+self.data = SerializedDAG.to_dict(dag)
+self.last_updated = timezone.utcnow()
+
+@staticmethod
+def dag_fileloc_hash(full_filepath: str) -> int:
+Hashing file location for indexing.
+
+:param full_filepath: full filepath of DAG file
+:return: hashed full_filepath
+"""
+# hashing is needed because the length of fileloc is 2000 as an 
Airflow convention,
+# which is over the limit of indexing. If we can reduce the length of 
fileloc, then
+# hashing is not needed.
+return int.from_bytes(
+hashlib.sha1(full_filepath.encode('utf-8')).digest()[-2:], 
byteorder='big', signed=False)
+
+@classmethod
+@db.provide_session
+def write_dag(cls, dag: 'DAG', min_update_interval: Optional[int] = None, 
session=None):
+"""Serializes a DAG and writes it into database.
+
+:param dag: a DAG to be written into database
+:param min_update_interval: minimal interval in seconds to update 
serialized DAG
+:param session: ORM Session
+"""
+log.debug("Writing DAG: %s to the DB", dag)
 
 Review comment:
   ```suggestion
   log.debug("Writing DAG: %s to the DB", dag.dag_id)
   ```


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 

[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-08-12 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r313163517
 
 

 ##
 File path: airflow/dag/serialization/serialized_baseoperator.py
 ##
 @@ -45,6 +45,8 @@ def __init__(self, *args, **kwargs):
 self.ui_color = BaseOperator.ui_color
 self.ui_fgcolor = BaseOperator.ui_fgcolor
 self.template_fields = BaseOperator.template_fields
+# Not None for SubDagOperator.
 
 Review comment:
   Can we add validation here to ensure that this field is `None` if the the 
operator is not `SubDagOperator`?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [airflow] milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] Persisting serialized DAG in DB for webserver scalability

2019-08-08 Thread GitBox
milton0825 commented on a change in pull request #5743: [AIRFLOW-5088][AIP-24] 
Persisting serialized DAG in DB for webserver scalability
URL: https://github.com/apache/airflow/pull/5743#discussion_r312151765
 
 

 ##
 File path: airflow/migrations/versions/d38e04c12aa2_add_serialized_dag_table.py
 ##
 @@ -0,0 +1,50 @@
+#
+# 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 serialized_dag table
+
+Revision ID: d38e04c12aa2
+Revises: 6e96a59344a4
+Create Date: 2019-08-01 14:39:35.616417
+
+"""
+
+# revision identifiers, used by Alembic.
+revision = 'd38e04c12aa2'
+down_revision = '6e96a59344a4'
+branch_labels = None
+depends_on = None
+
+from alembic import op
+import sqlalchemy as sa
+
+
+def upgrade():
+op.create_table('serialized_dag',
 
 Review comment:
   Make sense


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:
us...@infra.apache.org


With regards,
Apache Git Services