github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/hadoopfilesystem_test.py (about)

     1  #
     2  # Licensed to the Apache Software Foundation (ASF) under one or more
     3  # contributor license agreements.  See the NOTICE file distributed with
     4  # this work for additional information regarding copyright ownership.
     5  # The ASF licenses this file to You under the Apache License, Version 2.0
     6  # (the "License"); you may not use this file except in compliance with
     7  # the License.  You may obtain a copy of the License at
     8  #
     9  #    http://www.apache.org/licenses/LICENSE-2.0
    10  #
    11  # Unless required by applicable law or agreed to in writing, software
    12  # distributed under the License is distributed on an "AS IS" BASIS,
    13  # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  # See the License for the specific language governing permissions and
    15  # limitations under the License.
    16  #
    17  
    18  """Unit tests for :class:`HadoopFileSystem`."""
    19  
    20  # pytype: skip-file
    21  
    22  import io
    23  import logging
    24  import posixpath
    25  import time
    26  import unittest
    27  
    28  from parameterized import parameterized_class
    29  
    30  from apache_beam.io import hadoopfilesystem as hdfs
    31  from apache_beam.io.filesystem import BeamIOError
    32  from apache_beam.options.pipeline_options import HadoopFileSystemOptions
    33  from apache_beam.options.pipeline_options import PipelineOptions
    34  
    35  
    36  class FakeFile(io.BytesIO):
    37    """File object for FakeHdfs"""
    38    __hash__ = None  # type: ignore[assignment]
    39  
    40    def __init__(self, path, mode='', type='FILE', time_ms=None):
    41      io.BytesIO.__init__(self)
    42      if time_ms is None:
    43        time_ms = int(time.time() * 1000)
    44      self.time_ms = time_ms
    45      self.stat = {'path': path, 'mode': mode, 'type': type}
    46      self.saved_data = None
    47  
    48    def __eq__(self, other):
    49      """Equality of two files. Timestamp not included in comparison"""
    50      return self.stat == other.stat and self.getvalue() == self.getvalue()
    51  
    52    def close(self):
    53      self.saved_data = self.getvalue()
    54      io.BytesIO.close(self)
    55  
    56    def __enter__(self):
    57      return self
    58  
    59    def __exit__(self, exc_type, exc_val, exc_tb):
    60      self.close()
    61  
    62    @property
    63    def size(self):
    64      if self.closed:  # pylint: disable=using-constant-test
    65        if self.saved_data is None:
    66          return 0
    67        return len(self.saved_data)
    68      return len(self.getvalue())
    69  
    70    def get_file_status(self):
    71      """Returns a partial WebHDFS FileStatus object."""
    72      return {
    73          hdfs._FILE_STATUS_PATH_SUFFIX: posixpath.basename(self.stat['path']),
    74          hdfs._FILE_STATUS_LENGTH: self.size,
    75          hdfs._FILE_STATUS_TYPE: self.stat['type'],
    76          hdfs._FILE_STATUS_UPDATED: self.time_ms
    77      }
    78  
    79    def get_file_checksum(self):
    80      """Returns a WebHDFS FileChecksum object."""
    81      return {
    82          hdfs._FILE_CHECKSUM_ALGORITHM: 'fake_algo',
    83          hdfs._FILE_CHECKSUM_BYTES: 'checksum_byte_sequence',
    84          hdfs._FILE_CHECKSUM_LENGTH: 5,
    85      }
    86  
    87  
    88  class FakeHdfsError(Exception):
    89    """Generic error for FakeHdfs methods."""
    90  
    91  
    92  class FakeHdfs(object):
    93    """Fake implementation of ``hdfs.Client``."""
    94    def __init__(self):
    95      self.files = {}
    96  
    97    def write(self, path):
    98      if self.status(path, strict=False) is not None:
    99        raise FakeHdfsError('Path already exists: %s' % path)
   100  
   101      new_file = FakeFile(path, 'wb')
   102      self.files[path] = new_file
   103      return new_file
   104  
   105    def read(self, path, offset=0, length=None):
   106      old_file = self.files.get(path, None)
   107      if old_file is None:
   108        raise FakeHdfsError('Path not found: %s' % path)
   109      if old_file.stat['type'] == 'DIRECTORY':
   110        raise FakeHdfsError('Cannot open a directory: %s' % path)
   111      if not old_file.closed:
   112        raise FakeHdfsError('File already opened: %s' % path)
   113  
   114      # old_file is closed and can't be operated upon. Return a copy instead.
   115      new_file = FakeFile(path, 'rb')
   116      if old_file.saved_data:
   117        if length is None:
   118          new_file.write(old_file.saved_data)
   119        else:
   120          new_file.write(old_file.saved_data[:offset + length])
   121        new_file.seek(offset)
   122      return new_file
   123  
   124    def list(self, path, status=False):
   125      if not status:
   126        raise ValueError('status must be True')
   127      fs = self.status(path, strict=False)
   128      if (fs is not None and
   129          fs[hdfs._FILE_STATUS_TYPE] == hdfs._FILE_STATUS_TYPE_FILE):
   130        raise ValueError(
   131            'list must be called on a directory, got file: %s' % path)
   132  
   133      result = []
   134      for file in self.files.values():
   135        if file.stat['path'].startswith(path):
   136          fs = file.get_file_status()
   137          result.append((fs[hdfs._FILE_STATUS_PATH_SUFFIX], fs))
   138      return result
   139  
   140    def makedirs(self, path):
   141      self.files[path] = FakeFile(path, type='DIRECTORY')
   142  
   143    def status(self, path, strict=True):
   144      f = self.files.get(path)
   145      if f is None:
   146        if strict:
   147          raise FakeHdfsError('Path not found: %s' % path)
   148        else:
   149          return f
   150      return f.get_file_status()
   151  
   152    def delete(self, path, recursive=True):
   153      if not recursive:
   154        raise FakeHdfsError('Non-recursive mode not implemented')
   155  
   156      _ = self.status(path)
   157  
   158      for filepath in list(self.files):
   159        if filepath.startswith(path):
   160          del self.files[filepath]
   161  
   162    def walk(self, path):
   163      paths = [path]
   164      while paths:
   165        path = paths.pop()
   166        files = []
   167        dirs = []
   168        for full_path in self.files:
   169          if not full_path.startswith(path):
   170            continue
   171          short_path = posixpath.relpath(full_path, path)
   172          if '/' not in short_path:
   173            if self.status(full_path)[hdfs._FILE_STATUS_TYPE] == 'DIRECTORY':
   174              if short_path != '.':
   175                dirs.append(short_path)
   176            else:
   177              files.append(short_path)
   178  
   179        yield path, dirs, files
   180        paths = [posixpath.join(path, dir) for dir in dirs]
   181  
   182    def rename(self, path1, path2):
   183      if self.status(path1, strict=False) is None:
   184        raise FakeHdfsError('Path1 not found: %s' % path1)
   185  
   186      files_to_rename = [
   187          path for path in self.files
   188          if path == path1 or path.startswith(path1 + '/')
   189      ]
   190      for fullpath in files_to_rename:
   191        f = self.files.pop(fullpath)
   192        newpath = path2 + fullpath[len(path1):]
   193        f.stat['path'] = newpath
   194        self.files[newpath] = f
   195  
   196    def checksum(self, path):
   197      f = self.files.get(path, None)
   198      if f is None:
   199        raise FakeHdfsError('Path not found: %s' % path)
   200      return f.get_file_checksum()
   201  
   202  
   203  @parameterized_class(('full_urls', ), [(False, ), (True, )])
   204  class HadoopFileSystemTest(unittest.TestCase):
   205    def setUp(self):
   206      self._fake_hdfs = FakeHdfs()
   207      hdfs.hdfs.InsecureClient = (lambda *args, **kwargs: self._fake_hdfs)
   208      pipeline_options = PipelineOptions()
   209      hdfs_options = pipeline_options.view_as(HadoopFileSystemOptions)
   210      hdfs_options.hdfs_host = ''
   211      hdfs_options.hdfs_port = 0
   212      hdfs_options.hdfs_user = ''
   213  
   214      self.fs = hdfs.HadoopFileSystem(pipeline_options)
   215      self.fs._full_urls = self.full_urls
   216      if self.full_urls:
   217        self.tmpdir = 'hdfs://test_dir'
   218      else:
   219        self.tmpdir = 'hdfs://server/test_dir'
   220  
   221      for filename in ['old_file1', 'old_file2']:
   222        url = self.fs.join(self.tmpdir, filename)
   223        self.fs.create(url).close()
   224  
   225    def test_scheme(self):
   226      self.assertEqual(self.fs.scheme(), 'hdfs')
   227      self.assertEqual(hdfs.HadoopFileSystem.scheme(), 'hdfs')
   228  
   229    def test_parse_url(self):
   230      cases = [
   231          ('hdfs://', ('', '/'), False),
   232          ('hdfs://', None, True),
   233          ('hdfs://a', ('', '/a'), False),
   234          ('hdfs://a', ('a', '/'), True),
   235          ('hdfs://a/', ('', '/a/'), False),
   236          ('hdfs://a/', ('a', '/'), True),
   237          ('hdfs://a/b', ('', '/a/b'), False),
   238          ('hdfs://a/b', ('a', '/b'), True),
   239          ('hdfs://a/b/', ('', '/a/b/'), False),
   240          ('hdfs://a/b/', ('a', '/b/'), True),
   241          ('hdfs:/a/b', None, False),
   242          ('hdfs:/a/b', None, True),
   243          ('invalid', None, False),
   244          ('invalid', None, True),
   245      ]
   246      for url, expected, full_urls in cases:
   247        if self.full_urls != full_urls:
   248          continue
   249        try:
   250          result = self.fs._parse_url(url)
   251        except ValueError:
   252          self.assertIsNone(expected, msg=(url, expected, full_urls))
   253          continue
   254        self.assertEqual(expected, result, msg=(url, expected, full_urls))
   255  
   256    def test_url_join(self):
   257      self.assertEqual(
   258          'hdfs://tmp/path/to/file',
   259          self.fs.join('hdfs://tmp/path', 'to', 'file'))
   260      self.assertEqual(
   261          'hdfs://tmp/path/to/file', self.fs.join('hdfs://tmp/path', 'to/file'))
   262      self.assertEqual('hdfs://tmp/path/', self.fs.join('hdfs://tmp/path/', ''))
   263  
   264      if not self.full_urls:
   265        self.assertEqual('hdfs://bar', self.fs.join('hdfs://foo', '/bar'))
   266        self.assertEqual('hdfs://bar', self.fs.join('hdfs://foo/', '/bar'))
   267        with self.assertRaises(ValueError):
   268          self.fs.join('/no/scheme', 'file')
   269      else:
   270        self.assertEqual('hdfs://foo/bar', self.fs.join('hdfs://foo', '/bar'))
   271        self.assertEqual('hdfs://foo/bar', self.fs.join('hdfs://foo/', '/bar'))
   272  
   273    def test_url_split(self):
   274      self.assertEqual(('hdfs://tmp/path/to', 'file'),
   275                       self.fs.split('hdfs://tmp/path/to/file'))
   276      if not self.full_urls:
   277        self.assertEqual(('hdfs://', 'tmp'), self.fs.split('hdfs://tmp'))
   278        self.assertEqual(('hdfs://tmp', ''), self.fs.split('hdfs://tmp/'))
   279        self.assertEqual(('hdfs://tmp', 'a'), self.fs.split('hdfs://tmp/a'))
   280      else:
   281        self.assertEqual(('hdfs://tmp/', ''), self.fs.split('hdfs://tmp'))
   282        self.assertEqual(('hdfs://tmp/', ''), self.fs.split('hdfs://tmp/'))
   283        self.assertEqual(('hdfs://tmp/', 'a'), self.fs.split('hdfs://tmp/a'))
   284  
   285      self.assertEqual(('hdfs://tmp/a', ''), self.fs.split('hdfs://tmp/a/'))
   286      with self.assertRaisesRegex(ValueError, r'parse'):
   287        self.fs.split('tmp')
   288  
   289    def test_mkdirs(self):
   290      url = self.fs.join(self.tmpdir, 't1/t2')
   291      self.fs.mkdirs(url)
   292      self.assertTrue(self.fs.exists(url))
   293  
   294    def test_mkdirs_failed(self):
   295      url = self.fs.join(self.tmpdir, 't1/t2')
   296      self.fs.mkdirs(url)
   297  
   298      with self.assertRaises(IOError):
   299        self.fs.mkdirs(url)
   300  
   301    def test_match_file(self):
   302      expected_files = [
   303          self.fs.join(self.tmpdir, filename)
   304          for filename in ['old_file1', 'old_file2']
   305      ]
   306      match_patterns = expected_files
   307      result = self.fs.match(match_patterns)
   308      returned_files = [
   309          f.path for match_result in result for f in match_result.metadata_list
   310      ]
   311      self.assertCountEqual(expected_files, returned_files)
   312  
   313    def test_match_file_with_limits(self):
   314      expected_files = [
   315          self.fs.join(self.tmpdir, filename)
   316          for filename in ['old_file1', 'old_file2']
   317      ]
   318      result = self.fs.match([self.tmpdir + '/'], [1])[0]
   319      files = [f.path for f in result.metadata_list]
   320      self.assertEqual(len(files), 1)
   321      self.assertIn(files[0], expected_files)
   322  
   323    def test_match_file_with_zero_limit(self):
   324      result = self.fs.match([self.tmpdir + '/'], [0])[0]
   325      self.assertEqual(len(result.metadata_list), 0)
   326  
   327    def test_match_file_empty(self):
   328      url = self.fs.join(self.tmpdir, 'nonexistent_file')
   329      result = self.fs.match([url])[0]
   330      files = [f.path for f in result.metadata_list]
   331      self.assertEqual(files, [])
   332  
   333    def test_match_file_error(self):
   334      url = self.fs.join(self.tmpdir, 'old_file1')
   335      bad_url = 'bad_url'
   336      with self.assertRaisesRegex(BeamIOError,
   337                                  r'^Match operation failed .* %s' % bad_url):
   338        result = self.fs.match([bad_url, url])[0]
   339        files = [f.path for f in result.metadata_list]
   340        self.assertEqual(files, [self.fs._parse_url(url)])
   341  
   342    def test_match_directory(self):
   343      expected_files = [
   344          self.fs.join(self.tmpdir, filename)
   345          for filename in ['old_file1', 'old_file2']
   346      ]
   347  
   348      # Listing without a trailing '/' should return the directory itself and not
   349      # its contents. The fake HDFS client here has a "sparse" directory
   350      # structure, so listing without a '/' will return no results.
   351      result = self.fs.match([self.tmpdir + '/'])[0]
   352      files = [f.path for f in result.metadata_list]
   353      self.assertCountEqual(files, expected_files)
   354  
   355    def test_match_directory_trailing_slash(self):
   356      expected_files = [
   357          self.fs.join(self.tmpdir, filename)
   358          for filename in ['old_file1', 'old_file2']
   359      ]
   360  
   361      result = self.fs.match([self.tmpdir + '/'])[0]
   362      files = [f.path for f in result.metadata_list]
   363      self.assertCountEqual(files, expected_files)
   364  
   365    def test_create_success(self):
   366      url = self.fs.join(self.tmpdir, 'new_file')
   367      handle = self.fs.create(url)
   368      self.assertIsNotNone(handle)
   369      _, url = self.fs._parse_url(url)
   370      expected_file = FakeFile(url, 'wb')
   371      self.assertEqual(self._fake_hdfs.files[url], expected_file)
   372  
   373    def test_create_write_read_compressed(self):
   374      url = self.fs.join(self.tmpdir, 'new_file.gz')
   375  
   376      handle = self.fs.create(url)
   377      self.assertIsNotNone(handle)
   378      _, path = self.fs._parse_url(url)
   379      expected_file = FakeFile(path, 'wb')
   380      self.assertEqual(self._fake_hdfs.files[path], expected_file)
   381      data = b'abc' * 10
   382      handle.write(data)
   383      # Compressed data != original data
   384      self.assertNotEqual(data, self._fake_hdfs.files[path].getvalue())
   385      handle.close()
   386  
   387      handle = self.fs.open(url)
   388      read_data = handle.read(len(data))
   389      self.assertEqual(data, read_data)
   390      handle.close()
   391  
   392    def test_random_read_large_file(self):
   393      # this tests HdfsDownloader.get_range() works properly with
   394      # filesystemio.readinto when reading a file of size larger than the buffer.
   395      url = self.fs.join(self.tmpdir, 'read_length')
   396      handle = self.fs.create(url)
   397      data = b'test' * 10_000_000
   398      handle.write(data)
   399      handle.close()
   400  
   401      handle = self.fs.open(url)
   402      handle.seek(100)
   403      # read 3 bytes
   404      read_data = handle.read(3)
   405      self.assertEqual(data[100:103], read_data)
   406      # read 4 bytes
   407      read_data = handle.read(4)
   408      self.assertEqual(data[103:107], read_data)
   409  
   410    def test_open(self):
   411      url = self.fs.join(self.tmpdir, 'old_file1')
   412      handle = self.fs.open(url)
   413      expected_data = b''
   414      data = handle.read()
   415      self.assertEqual(data, expected_data)
   416  
   417    def test_open_bad_path(self):
   418      with self.assertRaises(FakeHdfsError):
   419        self.fs.open(self.fs.join(self.tmpdir, 'nonexistent/path'))
   420  
   421    def _cmpfiles(self, url1, url2):
   422      with self.fs.open(url1) as f1:
   423        with self.fs.open(url2) as f2:
   424          data1 = f1.read()
   425          data2 = f2.read()
   426          return data1 == data2
   427  
   428    def test_copy_file(self):
   429      url1 = self.fs.join(self.tmpdir, 'new_file1')
   430      url2 = self.fs.join(self.tmpdir, 'new_file2')
   431      url3 = self.fs.join(self.tmpdir, 'new_file3')
   432      with self.fs.create(url1) as f1:
   433        f1.write(b'Hello')
   434      self.fs.copy([url1, url1], [url2, url3])
   435      self.assertTrue(self._cmpfiles(url1, url2))
   436      self.assertTrue(self._cmpfiles(url1, url3))
   437  
   438    def test_copy_file_overwrite_error(self):
   439      url1 = self.fs.join(self.tmpdir, 'new_file1')
   440      url2 = self.fs.join(self.tmpdir, 'new_file2')
   441      with self.fs.create(url1) as f1:
   442        f1.write(b'Hello')
   443      with self.fs.create(url2) as f2:
   444        f2.write(b'nope')
   445      with self.assertRaisesRegex(BeamIOError,
   446                                  r'already exists.*%s' %
   447                                  posixpath.basename(url2)):
   448        self.fs.copy([url1], [url2])
   449  
   450    def test_copy_file_error(self):
   451      url1 = self.fs.join(self.tmpdir, 'new_file1')
   452      url2 = self.fs.join(self.tmpdir, 'new_file2')
   453      url3 = self.fs.join(self.tmpdir, 'new_file3')
   454      url4 = self.fs.join(self.tmpdir, 'new_file4')
   455      with self.fs.create(url3) as f:
   456        f.write(b'Hello')
   457      with self.assertRaisesRegex(BeamIOError,
   458                                  r'^Copy operation failed .*%s.*%s.* not found' %
   459                                  (url1, url2)):
   460        self.fs.copy([url1, url3], [url2, url4])
   461      self.assertTrue(self._cmpfiles(url3, url4))
   462  
   463    def test_copy_directory(self):
   464      url_t1 = self.fs.join(self.tmpdir, 't1')
   465      url_t1_inner = self.fs.join(self.tmpdir, 't1/inner')
   466      url_t2 = self.fs.join(self.tmpdir, 't2')
   467      url_t2_inner = self.fs.join(self.tmpdir, 't2/inner')
   468      self.fs.mkdirs(url_t1)
   469      self.fs.mkdirs(url_t1_inner)
   470      self.fs.mkdirs(url_t2)
   471  
   472      url1 = self.fs.join(url_t1_inner, 'f1')
   473      url2 = self.fs.join(url_t2_inner, 'f1')
   474      with self.fs.create(url1) as f:
   475        f.write(b'Hello')
   476  
   477      self.fs.copy([url_t1], [url_t2])
   478      self.assertTrue(self._cmpfiles(url1, url2))
   479  
   480    def test_copy_directory_overwrite_error(self):
   481      url_t1 = self.fs.join(self.tmpdir, 't1')
   482      url_t1_inner = self.fs.join(self.tmpdir, 't1/inner')
   483      url_t2 = self.fs.join(self.tmpdir, 't2')
   484      url_t2_inner = self.fs.join(self.tmpdir, 't2/inner')
   485      self.fs.mkdirs(url_t1)
   486      self.fs.mkdirs(url_t1_inner)
   487      self.fs.mkdirs(url_t2)
   488      self.fs.mkdirs(url_t2_inner)
   489  
   490      url1 = self.fs.join(url_t1, 'f1')
   491      url1_inner = self.fs.join(url_t1_inner, 'f2')
   492      url2 = self.fs.join(url_t2, 'f1')
   493      unused_url2_inner = self.fs.join(url_t2_inner, 'f2')
   494      url3_inner = self.fs.join(url_t2_inner, 'f3')
   495      for url in [url1, url1_inner, url3_inner]:
   496        with self.fs.create(url) as f:
   497          f.write(b'Hello')
   498      with self.fs.create(url2) as f:
   499        f.write(b'nope')
   500  
   501      with self.assertRaisesRegex(BeamIOError, r'already exists'):
   502        self.fs.copy([url_t1], [url_t2])
   503  
   504    def test_rename_file(self):
   505      url1 = self.fs.join(self.tmpdir, 'f1')
   506      url2 = self.fs.join(self.tmpdir, 'f2')
   507      with self.fs.create(url1) as f:
   508        f.write(b'Hello')
   509  
   510      self.fs.rename([url1], [url2])
   511      self.assertFalse(self.fs.exists(url1))
   512      self.assertTrue(self.fs.exists(url2))
   513  
   514    def test_rename_file_error(self):
   515      url1 = self.fs.join(self.tmpdir, 'f1')
   516      url2 = self.fs.join(self.tmpdir, 'f2')
   517      url3 = self.fs.join(self.tmpdir, 'f3')
   518      url4 = self.fs.join(self.tmpdir, 'f4')
   519      with self.fs.create(url3) as f:
   520        f.write(b'Hello')
   521  
   522      with self.assertRaisesRegex(BeamIOError,
   523                                  r'^Rename operation failed .*%s.*%s' %
   524                                  (url1, url2)):
   525        self.fs.rename([url1, url3], [url2, url4])
   526      self.assertFalse(self.fs.exists(url3))
   527      self.assertTrue(self.fs.exists(url4))
   528  
   529    def test_rename_directory(self):
   530      url_t1 = self.fs.join(self.tmpdir, 't1')
   531      url_t2 = self.fs.join(self.tmpdir, 't2')
   532      self.fs.mkdirs(url_t1)
   533      url1 = self.fs.join(url_t1, 'f1')
   534      url2 = self.fs.join(url_t2, 'f1')
   535      with self.fs.create(url1) as f:
   536        f.write(b'Hello')
   537  
   538      self.fs.rename([url_t1], [url_t2])
   539      self.assertFalse(self.fs.exists(url_t1))
   540      self.assertTrue(self.fs.exists(url_t2))
   541      self.assertFalse(self.fs.exists(url1))
   542      self.assertTrue(self.fs.exists(url2))
   543  
   544    def test_exists(self):
   545      url1 = self.fs.join(self.tmpdir, 'old_file1')
   546      url2 = self.fs.join(self.tmpdir, 'nonexistent')
   547      self.assertTrue(self.fs.exists(url1))
   548      self.assertFalse(self.fs.exists(url2))
   549  
   550    def test_size(self):
   551      url = self.fs.join(self.tmpdir, 'f1')
   552      with self.fs.create(url) as f:
   553        f.write(b'Hello')
   554      self.assertEqual(5, self.fs.size(url))
   555  
   556    def test_checksum(self):
   557      url = self.fs.join(self.tmpdir, 'f1')
   558      with self.fs.create(url) as f:
   559        f.write(b'Hello')
   560      self.assertEqual(
   561          'fake_algo-5-checksum_byte_sequence', self.fs.checksum(url))
   562  
   563    def test_last_updated(self):
   564      url = self.fs.join(self.tmpdir, 'f1')
   565      with self.fs.create(url) as f:
   566        f.write(b'Hello')
   567      # The time difference should be tiny for the mock hdfs.
   568      # A loose tolerance is for the consideration of real web hdfs.
   569      tolerance = 5 * 60  # 5 mins
   570      result = self.fs.last_updated(url)
   571      self.assertAlmostEqual(result, time.time(), delta=tolerance)
   572  
   573    def test_delete_file(self):
   574      url = self.fs.join(self.tmpdir, 'old_file1')
   575  
   576      self.assertTrue(self.fs.exists(url))
   577      self.fs.delete([url])
   578      self.assertFalse(self.fs.exists(url))
   579  
   580    def test_delete_dir(self):
   581      url_t1 = self.fs.join(self.tmpdir, 'new_dir1')
   582      url_t2 = self.fs.join(url_t1, 'new_dir2')
   583      url1 = self.fs.join(url_t2, 'new_file1')
   584      url2 = self.fs.join(url_t2, 'new_file2')
   585      self.fs.mkdirs(url_t1)
   586      self.fs.mkdirs(url_t2)
   587      self.fs.create(url1).close()
   588      self.fs.create(url2).close()
   589  
   590      self.assertTrue(self.fs.exists(url1))
   591      self.assertTrue(self.fs.exists(url2))
   592      self.fs.delete([url_t1])
   593      self.assertFalse(self.fs.exists(url_t1))
   594      self.assertFalse(self.fs.exists(url_t2))
   595      self.assertFalse(self.fs.exists(url2))
   596      self.assertFalse(self.fs.exists(url1))
   597  
   598    def test_delete_error(self):
   599      url1 = self.fs.join(self.tmpdir, 'nonexistent')
   600      url2 = self.fs.join(self.tmpdir, 'old_file1')
   601  
   602      self.assertTrue(self.fs.exists(url2))
   603      _, path1 = self.fs._parse_url(url1)
   604      with self.assertRaisesRegex(BeamIOError,
   605                                  r'^Delete operation failed .* %s' % path1):
   606        self.fs.delete([url1, url2])
   607      self.assertFalse(self.fs.exists(url2))
   608  
   609  
   610  class HadoopFileSystemRuntimeValueProviderTest(unittest.TestCase):
   611    """Tests pipeline_options, in the form of a
   612    RuntimeValueProvider.runtime_options object."""
   613    def setUp(self):
   614      self._fake_hdfs = FakeHdfs()
   615      hdfs.hdfs.InsecureClient = (lambda *args, **kwargs: self._fake_hdfs)
   616  
   617    def test_dict_options(self):
   618      pipeline_options = {
   619          'hdfs_host': '',
   620          'hdfs_port': 0,
   621          'hdfs_user': '',
   622      }
   623  
   624      self.fs = hdfs.HadoopFileSystem(pipeline_options=pipeline_options)
   625      self.assertFalse(self.fs._full_urls)
   626  
   627    def test_dict_options_missing(self):
   628      with self.assertRaisesRegex(ValueError, r'hdfs_host'):
   629        self.fs = hdfs.HadoopFileSystem(
   630            pipeline_options={
   631                'hdfs_port': 0,
   632                'hdfs_user': '',
   633            })
   634  
   635      with self.assertRaisesRegex(ValueError, r'hdfs_port'):
   636        self.fs = hdfs.HadoopFileSystem(
   637            pipeline_options={
   638                'hdfs_host': '',
   639                'hdfs_user': '',
   640            })
   641  
   642      with self.assertRaisesRegex(ValueError, r'hdfs_user'):
   643        self.fs = hdfs.HadoopFileSystem(
   644            pipeline_options={
   645                'hdfs_host': '',
   646                'hdfs_port': 0,
   647            })
   648  
   649    def test_dict_options_full_urls(self):
   650      pipeline_options = {
   651          'hdfs_host': '',
   652          'hdfs_port': 0,
   653          'hdfs_user': '',
   654          'hdfs_full_urls': 'invalid',
   655      }
   656  
   657      with self.assertRaisesRegex(ValueError, r'hdfs_full_urls'):
   658        self.fs = hdfs.HadoopFileSystem(pipeline_options=pipeline_options)
   659  
   660      pipeline_options['hdfs_full_urls'] = True
   661      self.fs = hdfs.HadoopFileSystem(pipeline_options=pipeline_options)
   662      self.assertTrue(self.fs._full_urls)
   663  
   664  
   665  if __name__ == '__main__':
   666    logging.getLogger().setLevel(logging.INFO)
   667    unittest.main()