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 2017/12/13 19:45:05 UTC

incubator-airflow git commit: [AIRFLOW-1687] fix fernet error without encryption

Repository: incubator-airflow
Updated Branches:
  refs/heads/master 1d7fac6e0 -> 3aa7ffa64


[AIRFLOW-1687] fix fernet error without encryption

Closes #2668 from TrevorEdwards/airflow-1687


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

Branch: refs/heads/master
Commit: 3aa7ffa64e0ae0b40d058e781d9a7ca8427ecbb7
Parents: 1d7fac6
Author: Trevor Edwards <tr...@google.com>
Authored: Wed Dec 13 20:44:44 2017 +0100
Committer: Bolke de Bruin <bo...@xs4all.nl>
Committed: Wed Dec 13 20:44:44 2017 +0100

----------------------------------------------------------------------
 airflow/models.py |  2 +-
 tests/models.py   | 27 +++++++++++++++++++++++++--
 2 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/3aa7ffa6/airflow/models.py
----------------------------------------------------------------------
diff --git a/airflow/models.py b/airflow/models.py
index a89da76..d69bc57 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -108,7 +108,7 @@ def get_fernet():
         raise AirflowException('Failed to import Fernet, it may not be installed')
     try:
         return Fernet(configuration.get('core', 'FERNET_KEY').encode('utf-8'))
-    except ValueError as ve:
+    except (ValueError, TypeError) as ve:
         raise AirflowException("Could not create Fernet object: {}".format(ve))
 
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/3aa7ffa6/tests/models.py
----------------------------------------------------------------------
diff --git a/tests/models.py b/tests/models.py
index a2433ab..3bab3cf 100644
--- a/tests/models.py
+++ b/tests/models.py
@@ -24,7 +24,7 @@ import pendulum
 import unittest
 import time
 
-from airflow import models, settings, AirflowException
+from airflow import configuration, models, settings, AirflowException
 from airflow.exceptions import AirflowSkipException
 from airflow.jobs import BackfillJob
 from airflow.models import DAG, TaskInstance as TI
@@ -32,6 +32,7 @@ from airflow.models import State as ST
 from airflow.models import DagModel, DagStat
 from airflow.models import clear_task_instances
 from airflow.models import XCom
+from airflow.models import Connection
 from airflow.operators.dummy_operator import DummyOperator
 from airflow.operators.bash_operator import BashOperator
 from airflow.operators.python_operator import PythonOperator
@@ -82,7 +83,6 @@ class DagTest(unittest.TestCase):
     def test_dag_as_context_manager(self):
         """
         Test DAG as a context manager.
-
         When used as a context manager, Operators are automatically added to
         the DAG (unless they specifiy a different DAG)
         """
@@ -1574,3 +1574,26 @@ class ClearTasksTest(unittest.TestCase):
 
         for result in results:
             self.assertEqual(result.value, json_obj)
+
+class ConnectionTest(unittest.TestCase):
+    @patch.object(configuration, 'get')
+    def test_connection_extra_no_encryption(self, mock_get):
+        """
+        Tests extras on a new connection without encryption. The fernet key
+        is set to a non-base64-encoded string and the extra is stored without
+        encryption.
+        """
+        mock_get.return_value = 'cryptography_not_found_storing_passwords_in_plain_text'
+        test_connection = Connection(extra='testextra')
+        self.assertEqual(test_connection.extra, 'testextra')
+
+    @patch.object(configuration, 'get')
+    def test_connection_extra_with_encryption(self, mock_get):
+        """
+        Tests extras on a new connection with encryption. The fernet key
+        is set to a base64 encoded string and the extra is encrypted.
+        """
+        # 'dGVzdA==' is base64 encoded 'test'
+        mock_get.return_value = 'dGVzdA=='
+        test_connection = Connection(extra='testextra')
+        self.assertEqual(test_connection.extra, 'testextra')