Skip to content

Commit

Permalink
[AIRFLOW-6396] Use tempfile.TemporaryDirectory instead of custom one (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
turbaszek authored Dec 31, 2019
1 parent da2a617 commit 7176a83
Show file tree
Hide file tree
Showing 14 changed files with 33 additions and 36 deletions.
10 changes: 10 additions & 0 deletions UPDATING.md
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,16 @@ https://developers.google.com/style/inclusive-documentation
-->

### Remove airflow.utils.file.TemporaryDirectory

Since Airflow dropped support for Python < 3.5 there's no need to have this custom
implementation of `TemporaryDirectory` because the same functionality is provided by
`tempfile.TemporaryDirectory`.

Now users instead of `import from airflow.utils.files import TemporaryDirectory` should
do `from tempfile import TemporaryDirectory`. Both context managers provide the same
interface, thus no additional changes should be required.

### Change python3 as Dataflow Hooks/Operators default interpreter

Now the `py_interpreter` argument for DataFlow Hooks/Operators has been changed from python2 to python3.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@
# under the License.

import os
from tempfile import TemporaryDirectory

import unicodecsv as csv

from airflow.contrib.hooks.azure_data_lake_hook import AzureDataLakeHook
from airflow.hooks.oracle_hook import OracleHook
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory


class OracleToAzureDataLakeTransfer(BaseOperator):
Expand Down
3 changes: 1 addition & 2 deletions airflow/contrib/sensors/bash_sensor.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,10 @@

import os
from subprocess import PIPE, STDOUT, Popen
from tempfile import NamedTemporaryFile, gettempdir
from tempfile import NamedTemporaryFile, TemporaryDirectory, gettempdir

from airflow.sensors.base_sensor_operator import BaseSensorOperator
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory


class BashSensor(BaseSensorOperator):
Expand Down
3 changes: 1 addition & 2 deletions airflow/hooks/hive_hooks.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,14 @@
import subprocess
import time
from collections import OrderedDict
from tempfile import NamedTemporaryFile
from tempfile import NamedTemporaryFile, TemporaryDirectory

import unicodecsv as csv

from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.hooks.base_hook import BaseHook
from airflow.security import utils
from airflow.utils.file import TemporaryDirectory
from airflow.utils.helpers import as_flattened_list
from airflow.utils.operator_helpers import AIRFLOW_VAR_NAME_FORMAT_MAPPING

Expand Down
3 changes: 1 addition & 2 deletions airflow/hooks/pig_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,10 @@
# under the License.

import subprocess
from tempfile import NamedTemporaryFile
from tempfile import NamedTemporaryFile, TemporaryDirectory

from airflow.exceptions import AirflowException
from airflow.hooks.base_hook import BaseHook
from airflow.utils.file import TemporaryDirectory


class PigCliHook(BaseHook):
Expand Down
3 changes: 1 addition & 2 deletions airflow/operators/bash_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,12 @@
import os
import signal
from subprocess import PIPE, STDOUT, Popen
from tempfile import gettempdir
from tempfile import TemporaryDirectory, gettempdir
from typing import Dict, Optional

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory
from airflow.utils.operator_helpers import context_to_airflow_vars


Expand Down
2 changes: 1 addition & 1 deletion airflow/operators/docker_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
"""
import ast
import json
from tempfile import TemporaryDirectory
from typing import Dict, Iterable, List, Optional, Union

from docker import APIClient, tls
Expand All @@ -29,7 +30,6 @@
from airflow.hooks.docker_hook import DockerHook
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory


# pylint: disable=too-many-instance-attributes
Expand Down
2 changes: 1 addition & 1 deletion airflow/operators/python_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import types
from inspect import signature
from itertools import islice
from tempfile import TemporaryDirectory
from textwrap import dedent
from typing import Callable, Dict, Iterable, List, Optional

Expand All @@ -33,7 +34,6 @@
from airflow.exceptions import AirflowException
from airflow.models import BaseOperator, SkipMixin
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory
from airflow.utils.operator_helpers import context_to_airflow_vars


Expand Down
3 changes: 1 addition & 2 deletions airflow/operators/s3_to_hive_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import gzip
import os
import tempfile
from tempfile import NamedTemporaryFile
from tempfile import NamedTemporaryFile, TemporaryDirectory
from typing import Dict, Optional, Union

from airflow.exceptions import AirflowException
Expand All @@ -34,7 +34,6 @@
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
from airflow.utils.compression import uncompress_file
from airflow.utils.decorators import apply_defaults
from airflow.utils.file import TemporaryDirectory


class S3ToHiveTransfer(BaseOperator): # pylint: disable=too-many-instance-attributes
Expand Down
24 changes: 8 additions & 16 deletions airflow/utils/file.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,30 +17,22 @@
# specific language governing permissions and limitations
# under the License.

import errno
import os
import re
import shutil
import zipfile
from contextlib import contextmanager
from tempfile import mkdtemp
from typing import Dict, List, Optional, Pattern

from airflow import LoggingMixin, conf


@contextmanager
def TemporaryDirectory(suffix='', prefix=None, dir=None):
name = mkdtemp(suffix=suffix, prefix=prefix, dir=dir)
try:
yield name
finally:
try:
shutil.rmtree(name)
except OSError as e:
# ENOENT - no such file or directory
if e.errno != errno.ENOENT:
raise e
def TemporaryDirectory(*args, **kwargs):
import warnings
from tempfile import TemporaryDirectory as TmpDir
warnings.warn(
"This function is deprecated. Please use `tempfile.TemporaryDirectory`",
DeprecationWarning, stacklevel=2
)
return TmpDir(*args, **kwargs)


def mkdirs(path, mode):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@

import os
import unittest
from tempfile import TemporaryDirectory

import mock
import unicodecsv as csv
from mock import MagicMock

from airflow.contrib.operators.oracle_to_azure_data_lake_transfer import OracleToAzureDataLakeTransfer
from airflow.utils.file import TemporaryDirectory


class TestOracleToAzureDataLakeTransfer(unittest.TestCase):
Expand Down
2 changes: 1 addition & 1 deletion tests/gcp/operators/test_cloud_build_system_helper.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,9 @@
"""
import argparse
import os
from tempfile import TemporaryDirectory
from urllib.parse import urlparse

from airflow.utils.file import TemporaryDirectory
from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor
from tests.gcp.utils.gcp_authenticator import GCP_CLOUD_BUILD_KEY, GcpAuthenticator

Expand Down
8 changes: 4 additions & 4 deletions tests/operators/test_docker_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,11 @@


class TestDockerOperator(unittest.TestCase):
@mock.patch('airflow.utils.file.mkdtemp')
@mock.patch('airflow.operators.docker_operator.TemporaryDirectory')
@mock.patch('airflow.operators.docker_operator.APIClient')
def test_execute(self, client_class_mock, mkdtemp_mock):
def test_execute(self, client_class_mock, tempdir_mock):
host_config = mock.Mock()
mkdtemp_mock.return_value = '/mkdtemp'
tempdir_mock.return_value.__enter__.return_value = '/mkdtemp'

client_mock = mock.Mock(spec=APIClient)
client_mock.create_container.return_value = {'Id': 'some_id'}
Expand Down Expand Up @@ -82,7 +82,7 @@ def test_execute(self, client_class_mock, mkdtemp_mock):
auto_remove=False,
dns=None,
dns_search=None)
mkdtemp_mock.assert_called_once_with(dir='/host/airflow', prefix='airflowtmp', suffix='')
tempdir_mock.assert_called_once_with(dir='/host/airflow', prefix='airflowtmp')
client_mock.images.assert_called_once_with(name='ubuntu:latest')
client_mock.attach.assert_called_once_with(container='some_id', stdout=True,
stderr=True, stream=True)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@
# specific language governing permissions and limitations
# under the License.
import os
from tempfile import TemporaryDirectory

from airflow.utils.file import TemporaryDirectory
from tests.contrib.utils.logging_command_executor import LoggingCommandExecutor


Expand Down

0 comments on commit 7176a83

Please sign in to comment.