From a38e8336c4659073d145e20b9cc9405d67a5340e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 16 Sep 2018 23:31:02 +0200 Subject: [PATCH] [AIRFLOW-3076] Remove preloading of MySQL testdata One of the things for tests is being self contained. This means that it should not depend on anything external, such as loading data. This PR will use the setUp and tearDown to load the data into MySQL and remove it afterwards. This removes the actual bash mysql commands and will make it easier to dockerize the whole testsuite in the future --- airflow/utils/db.py | 6 +- scripts/ci/4-load-data.sh | 29 --------- scripts/ci/data/baby_names.csv | 54 ---------------- scripts/ci/data/mysql_schema.sql | 27 -------- tests/operators/operators.py | 105 ++++++++++++++++++++++++------- tests/www/test_views.py | 2 +- tox.ini | 1 - 7 files changed, 86 insertions(+), 138 deletions(-) delete mode 100755 scripts/ci/4-load-data.sh delete mode 100644 scripts/ci/data/baby_names.csv delete mode 100644 scripts/ci/data/mysql_schema.sql diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b57b8cf92b3cb..5b457382aadff 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -96,11 +96,6 @@ def initdb(rbac=False): conn_id='airflow_db', conn_type='mysql', host='mysql', login='root', password='', schema='airflow')) - merge_conn( - models.Connection( - conn_id='airflow_ci', conn_type='mysql', - host='mysql', login='root', extra="{\"local_infile\": true}", - schema='airflow_ci')) merge_conn( models.Connection( conn_id='beeline_default', conn_type='beeline', port="10000", @@ -146,6 +141,7 @@ def initdb(rbac=False): models.Connection( conn_id='mysql_default', conn_type='mysql', login='root', + schema='airflow', host='mysql')) merge_conn( models.Connection( diff --git a/scripts/ci/4-load-data.sh b/scripts/ci/4-load-data.sh deleted file mode 100755 index 7935482be075e..0000000000000 --- a/scripts/ci/4-load-data.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash -# 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. * - -set -exuo pipefail - -DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) -DATA_DIR="${DIR}/data" -DATA_FILE="${DATA_DIR}/baby_names.csv" -DATABASE=airflow_ci -HOST=mysql - -mysqladmin -h ${HOST} -u root create ${DATABASE} -mysql -h ${HOST} -u root < ${DATA_DIR}/mysql_schema.sql -mysqlimport --local -h ${HOST} -u root --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 ${DATABASE} ${DATA_FILE} diff --git a/scripts/ci/data/baby_names.csv b/scripts/ci/data/baby_names.csv deleted file mode 100644 index b0ee22951e1ee..0000000000000 --- a/scripts/ci/data/baby_names.csv +++ /dev/null @@ -1,54 +0,0 @@ -"year","name","percent","sex" -1880,"John",0.081541,"boy" -1880,"William",0.080511,"boy" -1880,"James",0.050057,"boy" -1880,"Charles",0.045167,"boy" -1880,"George",0.043292,"boy" -1880,"Frank",0.02738,"boy" -1880,"Joseph",0.022229,"boy" -1880,"Thomas",0.021401,"boy" -1880,"Henry",0.020641,"boy" -1880,"Robert",0.020404,"boy" -1880,"Edward",0.019965,"boy" -1880,"Harry",0.018175,"boy" -1880,"Walter",0.014822,"boy" -1880,"Arthur",0.013504,"boy" -1880,"Fred",0.013251,"boy" -1880,"Albert",0.012609,"boy" -1880,"Samuel",0.008648,"boy" -1880,"David",0.007339,"boy" -1880,"Louis",0.006993,"boy" -1880,"Joe",0.006174,"boy" -1880,"Charlie",0.006165,"boy" -1880,"Clarence",0.006165,"boy" -1880,"Richard",0.006148,"boy" -1880,"Andrew",0.005439,"boy" -1880,"Daniel",0.00543,"boy" -1880,"Ernest",0.005194,"boy" -1880,"Will",0.004966,"boy" -1880,"Jesse",0.004805,"boy" -1880,"Oscar",0.004594,"boy" -1880,"Lewis",0.004366,"boy" -1880,"Peter",0.004189,"boy" -1880,"Benjamin",0.004138,"boy" -1880,"Frederick",0.004079,"boy" -1880,"Willie",0.00402,"boy" -1880,"Alfred",0.003961,"boy" -1880,"Sam",0.00386,"boy" -1880,"Roy",0.003716,"boy" -1880,"Herbert",0.003581,"boy" -1880,"Jacob",0.003412,"boy" -1880,"Tom",0.00337,"boy" -1880,"Elmer",0.00315,"boy" -1880,"Carl",0.003142,"boy" -1880,"Lee",0.003049,"boy" -1880,"Howard",0.003015,"boy" -1880,"Martin",0.003015,"boy" -1880,"Michael",0.00299,"boy" -1880,"Bert",0.002939,"boy" -1880,"Herman",0.002931,"boy" -1880,"Jim",0.002914,"boy" -1880,"Francis",0.002905,"boy" -1880,"Harvey",0.002905,"boy" -1880,"Earl",0.002829,"boy" -1880,"Eugene",0.00277,"boy" \ No newline at end of file diff --git a/scripts/ci/data/mysql_schema.sql b/scripts/ci/data/mysql_schema.sql deleted file mode 100644 index c2c60729d1468..0000000000000 --- a/scripts/ci/data/mysql_schema.sql +++ /dev/null @@ -1,27 +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. -*/ - -USE airflow_ci; - -CREATE TABLE IF NOT EXISTS baby_names ( - org_year integer(4), - baby_name VARCHAR(25), - rate FLOAT(7,6), - sex VARCHAR(4) -) \ No newline at end of file diff --git a/tests/operators/operators.py b/tests/operators/operators.py index ae953253cccf8..82870ac3eefbd 100644 --- a/tests/operators/operators.py +++ b/tests/operators/operators.py @@ -20,7 +20,6 @@ from __future__ import print_function from airflow import DAG, configuration, operators -from airflow.utils.tests import skipUnlessImported from airflow.utils import timezone from collections import OrderedDict @@ -38,13 +37,11 @@ TEST_DAG_ID = 'unit_test_dag' -@skipUnlessImported('airflow.operators.mysql_operator', 'MySqlOperator') class MySqlTest(unittest.TestCase): def setUp(self): configuration.load_test_config() args = { 'owner': 'airflow', - 'mysql_conn_id': 'airflow_db', 'start_date': DEFAULT_DATE } dag = DAG(TEST_DAG_ID, default_args=args) @@ -60,7 +57,6 @@ def test_mysql_operator_test(self): t = MySqlOperator( task_id='basic_mysql', sql=sql, - mysql_conn_id='airflow_db', dag=self.dag) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) @@ -72,8 +68,9 @@ def test_mysql_operator_test_multi(self): from airflow.operators.mysql_operator import MySqlOperator t = MySqlOperator( task_id='mysql_operator_test_multi', - mysql_conn_id='airflow_db', - sql=sql, dag=self.dag) + sql=sql, + dag=self.dag, + ) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) def test_mysql_hook_test_bulk_load(self): @@ -85,7 +82,7 @@ def test_mysql_hook_test_bulk_load(self): t.flush() from airflow.hooks.mysql_hook import MySqlHook - h = MySqlHook('airflow_ci') + h = MySqlHook('airflow_db') with h.get_conn() as c: c.execute(""" CREATE TABLE IF NOT EXISTS test_airflow ( @@ -100,7 +97,7 @@ def test_mysql_hook_test_bulk_load(self): def test_mysql_hook_test_bulk_dump(self): from airflow.hooks.mysql_hook import MySqlHook - hook = MySqlHook('airflow_ci') + hook = MySqlHook('airflow_db') priv = hook.get_first("SELECT @@global.secure_file_priv") if priv and priv[0]: # Confirm that no error occurs @@ -115,7 +112,7 @@ def test_mysql_hook_test_bulk_dump_mock(self, mock_get_conn): mock_get_conn.return_value.cursor.return_value.execute = mock_execute from airflow.hooks.mysql_hook import MySqlHook - hook = MySqlHook('airflow_ci') + hook = MySqlHook('airflow_db') table = "INFORMATION_SCHEMA.TABLES" tmp_file = "/path/to/output/file" hook.bulk_dump(table, tmp_file) @@ -167,7 +164,6 @@ def test_overwrite_schema(self): assert "Unknown database 'foobar'" in str(e) -@skipUnlessImported('airflow.operators.postgres_operator', 'PostgresOperator') class PostgresTest(unittest.TestCase): def setUp(self): configuration.load_test_config() @@ -259,7 +255,6 @@ def test_overwrite_schema(self): assert 'database "foobar" does not exist' in str(e) -@skipUnlessImported('airflow.operators.mysql_to_hive', 'MySqlToHiveTransfer') class TransferTests(unittest.TestCase): def setUp(self): configuration.load_test_config() @@ -267,6 +262,81 @@ def setUp(self): dag = DAG(TEST_DAG_ID, default_args=args) self.dag = dag + rows = [ + (1880, "John", 0.081541, "boy"), + (1880, "William", 0.080511, "boy"), + (1880, "James", 0.050057, "boy"), + (1880, "Charles", 0.045167, "boy"), + (1880, "George", 0.043292, "boy"), + (1880, "Frank", 0.02738, "boy"), + (1880, "Joseph", 0.022229, "boy"), + (1880, "Thomas", 0.021401, "boy"), + (1880, "Henry", 0.020641, "boy"), + (1880, "Robert", 0.020404, "boy"), + (1880, "Edward", 0.019965, "boy"), + (1880, "Harry", 0.018175, "boy"), + (1880, "Walter", 0.014822, "boy"), + (1880, "Arthur", 0.013504, "boy"), + (1880, "Fred", 0.013251, "boy"), + (1880, "Albert", 0.012609, "boy"), + (1880, "Samuel", 0.008648, "boy"), + (1880, "David", 0.007339, "boy"), + (1880, "Louis", 0.006993, "boy"), + (1880, "Joe", 0.006174, "boy"), + (1880, "Charlie", 0.006165, "boy"), + (1880, "Clarence", 0.006165, "boy"), + (1880, "Richard", 0.006148, "boy"), + (1880, "Andrew", 0.005439, "boy"), + (1880, "Daniel", 0.00543, "boy"), + (1880, "Ernest", 0.005194, "boy"), + (1880, "Will", 0.004966, "boy"), + (1880, "Jesse", 0.004805, "boy"), + (1880, "Oscar", 0.004594, "boy"), + (1880, "Lewis", 0.004366, "boy"), + (1880, "Peter", 0.004189, "boy"), + (1880, "Benjamin", 0.004138, "boy"), + (1880, "Frederick", 0.004079, "boy"), + (1880, "Willie", 0.00402, "boy"), + (1880, "Alfred", 0.003961, "boy"), + (1880, "Sam", 0.00386, "boy"), + (1880, "Roy", 0.003716, "boy"), + (1880, "Herbert", 0.003581, "boy"), + (1880, "Jacob", 0.003412, "boy"), + (1880, "Tom", 0.00337, "boy"), + (1880, "Elmer", 0.00315, "boy"), + (1880, "Carl", 0.003142, "boy"), + (1880, "Lee", 0.003049, "boy"), + (1880, "Howard", 0.003015, "boy"), + (1880, "Martin", 0.003015, "boy"), + (1880, "Michael", 0.00299, "boy"), + (1880, "Bert", 0.002939, "boy"), + (1880, "Herman", 0.002931, "boy"), + (1880, "Jim", 0.002914, "boy"), + (1880, "Francis", 0.002905, "boy"), + (1880, "Harvey", 0.002905, "boy"), + (1880, "Earl", 0.002829, "boy"), + (1880, "Eugene", 0.00277, "boy"), + ] + + from airflow.hooks.mysql_hook import MySqlHook + with MySqlHook().get_conn() as cur: + cur.execute(''' + CREATE TABLE IF NOT EXISTS baby_names ( + org_year integer(4), + baby_name VARCHAR(25), + rate FLOAT(7,6), + sex VARCHAR(4) + ) + ''') + + for row in rows: + cur.execute("INSERT INTO baby_names VALUES(%s, %s, %s, %s);", row) + + def tearDown(self): + from airflow.hooks.mysql_hook import MySqlHook + with MySqlHook().get_conn() as cur: + cur.execute("DROP TABLE IF EXISTS baby_names CASCADE;") + def test_clear(self): self.dag.clear( start_date=DEFAULT_DATE, @@ -277,7 +347,6 @@ def test_mysql_to_hive(self): sql = "SELECT * FROM baby_names LIMIT 1000;" t = MySqlToHiveTransfer( task_id='test_m2h', - mysql_conn_id='airflow_ci', hive_cli_conn_id='beeline_default', sql=sql, hive_table='test_mysql_to_hive', @@ -291,7 +360,6 @@ def test_mysql_to_hive_partition(self): sql = "SELECT * FROM baby_names LIMIT 1000;" t = MySqlToHiveTransfer( task_id='test_m2h', - mysql_conn_id='airflow_ci', hive_cli_conn_id='beeline_default', sql=sql, hive_table='test_mysql_to_hive_part', @@ -307,23 +375,21 @@ def test_mysql_to_hive_tblproperties(self): sql = "SELECT * FROM baby_names LIMIT 1000;" t = MySqlToHiveTransfer( task_id='test_m2h', - mysql_conn_id='airflow_ci', hive_cli_conn_id='beeline_default', sql=sql, hive_table='test_mysql_to_hive', recreate=True, delimiter=",", - tblproperties={'test_property':'test_value'}, + tblproperties={'test_property': 'test_value'}, dag=self.dag) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) @mock.patch('airflow.hooks.hive_hooks.HiveCliHook.load_file') def test_mysql_to_hive_type_conversion(self, mock_load_file): - mysql_conn_id = 'airflow_ci' mysql_table = 'test_mysql_to_hive' from airflow.hooks.mysql_hook import MySqlHook - m = MySqlHook(mysql_conn_id) + m = MySqlHook() try: with m.get_conn() as c: @@ -342,7 +408,6 @@ def test_mysql_to_hive_type_conversion(self, mock_load_file): from airflow.operators.mysql_to_hive import MySqlToHiveTransfer t = MySqlToHiveTransfer( task_id='test_m2h', - mysql_conn_id=mysql_conn_id, hive_cli_conn_id='beeline_default', sql="SELECT * FROM {}".format(mysql_table), hive_table='test_mysql_to_hive', @@ -363,12 +428,11 @@ def test_mysql_to_hive_type_conversion(self, mock_load_file): c.execute("DROP TABLE IF EXISTS {}".format(mysql_table)) def test_mysql_to_hive_verify_loaded_values(self): - mysql_conn_id = 'airflow_ci' mysql_table = 'test_mysql_to_hive' hive_table = 'test_mysql_to_hive' from airflow.hooks.mysql_hook import MySqlHook - m = MySqlHook(mysql_conn_id) + m = MySqlHook() try: minmax = ( @@ -409,7 +473,6 @@ def test_mysql_to_hive_verify_loaded_values(self): from airflow.operators.mysql_to_hive import MySqlToHiveTransfer t = MySqlToHiveTransfer( task_id='test_m2h', - mysql_conn_id=mysql_conn_id, hive_cli_conn_id='beeline_default', sql="SELECT * FROM {}".format(mysql_table), hive_table=hive_table, diff --git a/tests/www/test_views.py b/tests/www/test_views.py index fb7a2f3411870..8a7183fc4f600 100644 --- a/tests/www/test_views.py +++ b/tests/www/test_views.py @@ -67,7 +67,7 @@ def setUp(self): self.chart = { 'label': 'chart', 'owner': 'airflow', - 'conn_id': 'airflow_ci', + 'conn_id': 'airflow_db', } def tearDown(self): diff --git a/tox.ini b/tox.ini index 1ba7d84fea226..f07641f29b38a 100644 --- a/tox.ini +++ b/tox.ini @@ -60,7 +60,6 @@ commands = env_docker: {toxinidir}/scripts/ci/1-setup-env.sh env_docker: {toxinidir}/scripts/ci/2-setup-kdc.sh env_docker: {toxinidir}/scripts/ci/3-setup-databases.sh - env_docker: {toxinidir}/scripts/ci/4-load-data.sh {toxinidir}/scripts/ci/5-run-tests.sh [] {toxinidir}/scripts/ci/6-check-license.sh codecov -e TOXENV