You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by bo...@apache.org on 2016/11/13 22:02:23 UTC

[1/2] incubator-airflow git commit: [AIRFLOW-507] Use Travis' ubuntu trusty for CI

Repository: incubator-airflow
Updated Branches:
  refs/heads/master 6adb15ca0 -> 910c0ddd7


[AIRFLOW-507] Use Travis' ubuntu trusty for CI

Travis' ubuntu trusty provides a more up to date
environment for CI. It allows for better testing
by integration more services like kerberos and
celery. Also it comes closer to actual production
environments (e.g. MySQL 5.6).


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/d12ef6ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/d12ef6ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/d12ef6ff

Branch: refs/heads/master
Commit: d12ef6ffc00fdcf7c404f6680f6de2f39582b170
Parents: 6adb15c
Author: Bolke de Bruin <bo...@xs4all.nl>
Authored: Mon Sep 12 15:10:58 2016 +0200
Committer: Bolke de Bruin <bo...@xs4all.nl>
Committed: Sun Nov 13 22:43:16 2016 +0100

----------------------------------------------------------------------
 .travis.yml | 10 +++++++++-
 tox.ini     |  1 +
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/d12ef6ff/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 0b52183..6e844d7 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -11,7 +11,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-sudo: false
+sudo: true
+dist: trusty
 language: python
 jdk: oraclejdk7
 services:
@@ -23,6 +24,10 @@ addons:
       - slapd
       - ldap-utils
       - openssh-server
+      - mysql-server-5.6
+      - mysql-client-core-5.6
+      - mysql-client-5.6
+      - krb5-user
   postgresql: "9.2"
 python:
   - "2.7"
@@ -30,6 +35,9 @@ python:
 env:
   global:
     - TRAVIS_CACHE=$HOME/.travis_cache/
+    # Travis on google cloud engine has a global /etc/boto.cfg that
+    # does not work with python 3
+    - BOTO_CONFIG=/tmp/bogusvalue
   matrix:
     - TOX_ENV=py27-cdh-airflow_backend_mysql
     - TOX_ENV=py27-cdh-airflow_backend_sqlite

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/d12ef6ff/tox.ini
----------------------------------------------------------------------
diff --git a/tox.ini b/tox.ini
index 3003292..f447659 100644
--- a/tox.ini
+++ b/tox.ini
@@ -57,6 +57,7 @@ passenv =
     TRAVIS_CACHE
     TRAVIS_PULL_REQUEST
     PATH
+    BOTO_CONFIG
 commands =
   pip wheel -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -r scripts/ci/requirements.txt
   pip install --find-links={homedir}/.wheelhouse --no-index -r scripts/ci/requirements.txt


[2/2] incubator-airflow git commit: [AIRFLOW-504] Store fractional seconds in MySQL tables

Posted by bo...@apache.org.
[AIRFLOW-504] Store fractional seconds in MySQL tables

Both utcnow() and now() return fractional seconds. These
are sometimes used in primary_keys (eg. in task_instance).
If MySQL is not configured to store these fractional seconds
a primary key might fail (eg. at session.merge) resulting in
a duplicate entry being added or worse.

Postgres does store fractional seconds if left unconfigured,
sqlite needs to be examined.


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/910c0ddd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/910c0ddd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/910c0ddd

Branch: refs/heads/master
Commit: 910c0ddd78d45db1dc613359f3bb1d3ad37bdf74
Parents: d12ef6f
Author: Bolke de Bruin <bo...@xs4all.nl>
Authored: Mon Sep 12 15:19:17 2016 +0200
Committer: Bolke de Bruin <bo...@xs4all.nl>
Committed: Sun Nov 13 22:43:17 2016 +0100

----------------------------------------------------------------------
 ...f1_add_fractional_seconds_to_mysql_tables.py | 111 +++++++++++++++++++
 scripts/ci/load_data.sh                         |   2 +-
 tests/core.py                                   |  28 +++++
 3 files changed, 140 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/910c0ddd/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py
----------------------------------------------------------------------
diff --git a/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py
new file mode 100644
index 0000000..c1c6de3
--- /dev/null
+++ b/airflow/migrations/versions/4addfa1236f1_add_fractional_seconds_to_mysql_tables.py
@@ -0,0 +1,111 @@
+#
+# Licensed 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 fractional seconds to mysql tables
+
+Revision ID: 4addfa1236f1
+Revises: f2ca10b85618
+Create Date: 2016-09-11 13:39:18.592072
+
+"""
+
+# revision identifiers, used by Alembic.
+revision = '4addfa1236f1'
+down_revision = 'f2ca10b85618'
+branch_labels = None
+depends_on = None
+
+from alembic import op
+import sqlalchemy as sa
+from sqlalchemy.dialects import mysql
+from alembic import context
+
+
+def upgrade():
+    if context.config.get_main_option('sqlalchemy.url').startswith('mysql'):
+        op.alter_column(table_name='dag', column_name='last_scheduler_run', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='dag', column_name='last_pickled', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='dag', column_name='last_expired', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='dag_pickle', column_name='created_dttm', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='dag_run', column_name='execution_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='dag_run', column_name='start_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='dag_run', column_name='end_date', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='import_error', column_name='timestamp', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='job', column_name='start_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='job', column_name='end_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='job', column_name='latest_heartbeat', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='known_event', column_name='start_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='known_event', column_name='end_date', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='log', column_name='dttm', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='log', column_name='execution_date', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='sla_miss', column_name='execution_date', type_=mysql.DATETIME(fsp=6), nullable=False)
+        op.alter_column(table_name='sla_miss', column_name='timestamp', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='task_fail', column_name='execution_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='task_fail', column_name='start_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='task_fail', column_name='end_date', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='task_instance', column_name='execution_date', type_=mysql.DATETIME(fsp=6), nullable=False)
+        op.alter_column(table_name='task_instance', column_name='start_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='task_instance', column_name='end_date', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='task_instance', column_name='queued_dttm', type_=mysql.DATETIME(fsp=6))
+
+        op.alter_column(table_name='xcom', column_name='timestamp', type_=mysql.DATETIME(fsp=6))
+        op.alter_column(table_name='xcom', column_name='execution_date', type_=mysql.DATETIME(fsp=6))
+
+
+def downgrade():
+    if context.config.get_main_option('sqlalchemy.url').startswith('mysql'):
+        op.alter_column(table_name='dag', column_name='last_scheduler_run', type_=mysql.DATETIME())
+        op.alter_column(table_name='dag', column_name='last_pickled', type_=mysql.DATETIME())
+        op.alter_column(table_name='dag', column_name='last_expired', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='dag_pickle', column_name='created_dttm', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='dag_run', column_name='execution_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='dag_run', column_name='start_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='dag_run', column_name='end_date', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='import_error', column_name='timestamp', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='job', column_name='start_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='job', column_name='end_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='job', column_name='latest_heartbeat', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='known_event', column_name='start_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='known_event', column_name='end_date', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='log', column_name='dttm', type_=mysql.DATETIME())
+        op.alter_column(table_name='log', column_name='execution_date', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='sla_miss', column_name='execution_date', type_=mysql.DATETIME(), nullable=False)
+        op.alter_column(table_name='sla_miss', column_name='timestamp', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='task_fail', column_name='execution_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='task_fail', column_name='start_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='task_fail', column_name='end_date', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='task_instance', column_name='execution_date', type_=mysql.DATETIME(), nullable=False)
+        op.alter_column(table_name='task_instance', column_name='start_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='task_instance', column_name='end_date', type_=mysql.DATETIME())
+        op.alter_column(table_name='task_instance', column_name='queued_dttm', type_=mysql.DATETIME())
+
+        op.alter_column(table_name='xcom', column_name='timestamp', type_=mysql.DATETIME())
+        op.alter_column(table_name='xcom', column_name='execution_date', type_=mysql.DATETIME())

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/910c0ddd/scripts/ci/load_data.sh
----------------------------------------------------------------------
diff --git a/scripts/ci/load_data.sh b/scripts/ci/load_data.sh
index 0863d26..3422b07 100755
--- a/scripts/ci/load_data.sh
+++ b/scripts/ci/load_data.sh
@@ -24,5 +24,5 @@ DATABASE=airflow_ci
 
 mysqladmin -u root create ${DATABASE}
 mysql -u root < ${DATA_DIR}/mysql_schema.sql
-mysqlimport -u root --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 ${DATABASE} ${DATA_FILE}
+mysqlimport --local -u root --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 ${DATABASE} ${DATA_FILE}
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/910c0ddd/tests/core.py
----------------------------------------------------------------------
diff --git a/tests/core.py b/tests/core.py
index 912ee9f..effc63d 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -193,6 +193,34 @@ class CoreTest(unittest.TestCase):
         assert dag_run2 is None
         dag.clear()
 
+    def test_fractional_seconds(self):
+        """
+        Tests if fractional seconds are stored in the database
+        """
+        dag = DAG(TEST_DAG_ID + 'test_fractional_seconds')
+        dag.schedule_interval = '@once'
+        dag.add_task(models.BaseOperator(
+            task_id="faketastic",
+            owner='Also fake',
+            start_date=datetime(2015, 1, 2, 0, 0)))
+
+        start_date = datetime.now()
+
+        run = dag.create_dagrun(
+            run_id='test_' + start_date.isoformat(),
+            execution_date=start_date,
+            start_date=start_date,
+            state=State.RUNNING,
+            external_trigger=False
+        )
+
+        run.refresh_from_db()
+
+        self.assertEqual(start_date, run.execution_date,
+                         "dag run execution_date loses precision")
+        self.assertEqual(start_date, run.start_date,
+                         "dag run start_date loses precision ")
+
     def test_schedule_dag_start_end_dates(self):
         """
         Tests that an attempt to schedule a task after the Dag's end_date