def checksum(self, path): """Fetch checksum metadata of a file on the :class:`~apache_beam.io.filesystem.FileSystem`. Args: path: string path of a file. Returns: string containing checksum Raises: ``BeamIOError``: if path isn't a file or doesn't exist. """ try: return blobstorageio.BlobStorageIO().checksum(path) except Exception as e: # pylint: disable=broad-except raise BeamIOError("Checksum operation failed", {path, e})
def delete(paths): """Deletes files or directories at the provided paths. Directories will be deleted recursively. Args: paths: list of paths that give the file objects to be deleted Raises: ``BeamIOError`` if any of the delete operations fail """ if isinstance(paths, (str, unicode)): raise BeamIOError( 'Delete passed string argument instead of list: %s' % paths) if len(paths) == 0: return filesystem = FileSystems.get_filesystem(paths[0]) return filesystem.delete(paths)
def _list(self, dir_or_prefix): """List files in a location. Listing is non-recursive (for filesystems that support directories). Args: dir_or_prefix: (string) A directory or location prefix (for filesystems that don't have directories). Returns: Generator of ``FileMetadata`` objects. Raises: ``BeamIOError``: if listing fails, but not if no files were found. """ try: for path, (size, updated) in blobstorageio.BlobStorageIO() \ .list_prefix(dir_or_prefix, with_metadata=True).items(): yield FileMetadata(path, size, updated) except Exception as e: # pylint: disable=broad-except raise BeamIOError("List operation failed", {dir_or_prefix: e})
def process( self, file_metadata: Union[str, filesystem.FileMetadata], ) -> Iterable[ReadableFile]: metadata = (filesystem.FileMetadata(file_metadata, 0) if isinstance( file_metadata, str) else file_metadata) if ((metadata.path.endswith('/') or metadata.path.endswith('\\')) and self._skip_directories): return elif metadata.path.endswith('/') or metadata.path.endswith('\\'): raise BeamIOError( 'Directories are not allowed in ReadMatches transform.' 'Found %s.' % metadata.path) # TODO: Mime type? Other arguments? Maybe arguments passed in to transform? yield ReadableFile(metadata, self._compression)
def rename(self, source_file_names, destination_file_names): """Rename the files at the source list to the destination list. Source and destination lists should be of the same size. Args: source_file_names: List of file paths that need to be moved destination_file_names: List of destination_file_names for the files Raises: ``BeamIOError``: if any of the rename operations fail """ err_msg = ( "source_file_names and destination_file_names should " "be equal in length") assert len(source_file_names) == len(destination_file_names), err_msg gcs_batches = [] gcs_current_batch = [] for src, dest in zip(source_file_names, destination_file_names): gcs_current_batch.append((src, dest)) if len(gcs_current_batch) == self.CHUNK_SIZE: gcs_batches.append(gcs_current_batch) gcs_current_batch = [] if gcs_current_batch: gcs_batches.append(gcs_current_batch) # Execute GCS renames if any and return exceptions. exceptions = {} for batch in gcs_batches: copy_statuses = gcsio.GcsIO().copy_batch(batch) copy_succeeded = [] for src, dest, exception in copy_statuses: if exception: exceptions[(src, dest)] = exception else: copy_succeeded.append((src, dest)) delete_batch = [src for src, dest in copy_succeeded] delete_statuses = gcsio.GcsIO().delete_batch(delete_batch) for i, (src, exception) in enumerate(delete_statuses): dest = copy_succeeded[i][1] if exception: exceptions[(src, dest)] = exception if exceptions: raise BeamIOError("Rename operation failed", exceptions)
def metadata(self, path): """Fetch metadata fields of a file on the FileSystem. Args: path: string path of a file. Returns: :class:`~apache_beam.io.filesystem.FileMetadata`. Raises: ``BeamIOError``: if path isn't a file or doesn't exist. """ try: file_metadata = s3io.S3IO(options=self._options)._status(path) return FileMetadata(path, file_metadata['size'], file_metadata['last_updated']) except Exception as e: # pylint: disable=broad-except raise BeamIOError("Metadata operation failed", {path: e})
def metadata(self, url): """Fetch metadata fields of a file on the FileSystem. Args: url: string url of a file. Returns: :class:`~apache_beam.io.filesystem.FileMetadata`. Raises: ``BeamIOError``: if url doesn't exist. """ _, path = self._parse_url(url) status = self._hdfs_client.status(path, strict=False) if status is None: raise BeamIOError('File not found: %s' % url) return FileMetadata( url, status[_FILE_STATUS_LENGTH], status[_FILE_STATUS_UPDATED] / 1000.0)
def copy(self, source_file_names, destination_file_names): """Recursively copy the file tree from the source to the destination Args: source_file_names: list of source file objects that needs to be copied destination_file_names: list of destination of the new object Raises: ``BeamIOError`` if any of the copy operations fail """ err_msg = ("source_file_names and destination_file_names should " "be equal in length") assert len(source_file_names) == len(destination_file_names), err_msg def _copy_path(source, destination): """Recursively copy the file tree from the source to the destination """ try: if os.path.exists(destination): if os.path.isdir(destination): shutil.rmtree(destination) else: os.remove(destination) if os.path.isdir(source): shutil.copytree(source, destination) else: parent = os.path.dirname(destination) if not os.path.exists(parent): os.makedirs(parent) shutil.copy2(source, destination) except OSError as err: raise IOError(err) exceptions = {} for source, destination in zip(source_file_names, destination_file_names): try: _copy_path(source, destination) except Exception as e: # pylint: disable=broad-except exceptions[(source, destination)] = e if exceptions: raise BeamIOError("Copy operation failed", exceptions)
def delete(self, paths): """Deletes files or directories at the provided paths. Directories will be deleted recursively. Args: paths: list of paths that give the file objects to be deleted Raises: ``BeamIOError``: if any of the delete operations fail """ results = blobstorageio.BlobStorageIO().delete_paths(paths) # Retrieve exceptions. exceptions = { path: error for (path, error) in results.items() if error is not None } if exceptions: raise BeamIOError("Delete operation failed", exceptions)
def _list(self, dir_or_prefix): """List files in a location. Listing is non-recursive, for filesystems that support directories. Args: dir_or_prefix: (string) A directory or location prefix (for filesystems that don't have directories). Returns: Generator of ``FileMetadata`` objects. Raises: ``BeamIOError``: if listing fails, but not if no files were found. """ try: for path, size in iteritems(s3io.S3IO().list_prefix(dir_or_prefix)): yield FileMetadata(path, size) except Exception as e: # pylint: disable=broad-except raise BeamIOError("List operation failed", {dir_or_prefix: e})
def delete(self, paths): """Deletes files or directories at the provided paths. Directories will be deleted recursively. Args: paths: list of paths that give the file objects to be deleted Raises: ``BeamIOError``: if any of the delete operations fail """ def _delete_path(path): """Recursively delete the file or directory at the provided path. """ try: if os.path.isdir(path): shutil.rmtree(path) else: os.remove(path) except OSError as err: raise IOError(err) exceptions = {} def try_delete(path): try: _delete_path(path) except Exception as e: # pylint: disable=broad-except exceptions[path] = e for match_result in self.match(paths): metadata_list = match_result.metadata_list if not metadata_list: exceptions[match_result.pattern] = \ IOError('No files found to delete under: %s' % match_result.pattern) for metadata in match_result.metadata_list: try_delete(metadata.path) if exceptions: raise BeamIOError("Delete operation failed", exceptions)
def match(self, patterns, limits=None): """Find all matching paths to the pattern provided. Args: pattern: string for the file path pattern to match against limit: Maximum number of responses that need to be fetched Returns: list of ``MatchResult`` objects. Raises: ``BeamIOError`` if any of the pattern match operations fail """ if limits is None: limits = [None] * len(patterns) else: err_msg = "Patterns and limits should be equal in length" assert len(patterns) == len(limits), err_msg def _match(pattern, limit): """Find all matching paths to the pattern provided. """ if pattern.endswith('/'): pattern += '*' file_sizes = gcsio.GcsIO().size_of_files_in_glob(pattern, limit) metadata_list = [ FileMetadata(path, size) for path, size in file_sizes.iteritems() ] return MatchResult(pattern, metadata_list) exceptions = {} result = [] for pattern, limit in zip(patterns, limits): try: result.append(_match(pattern, limit)) except Exception as e: # pylint: disable=broad-except exceptions[pattern] = e if exceptions: raise BeamIOError("Match operation failed", exceptions) return result
def get_filesystem(path): """Get the correct filesystem for the specified path """ try: path_scheme = FileSystems.get_scheme(path) systems = [fs for fs in FileSystem.get_all_subclasses() if fs.scheme() == path_scheme] if len(systems) == 0: raise ValueError('Unable to get the Filesystem for path %s' % path) elif len(systems) == 1: # Pipeline options could come either from the Pipeline itself (using # direct runner), or via RuntimeValueProvider (other runners). options = (FileSystems._pipeline_options or RuntimeValueProvider.runtime_options) return systems[0](pipeline_options=options) else: raise ValueError('Found more than one filesystem for path %s' % path) except ValueError: raise except Exception as e: raise BeamIOError('Unable to get the Filesystem', {path: e})
def get_filesystem(path): """Get the correct filesystem for the specified path """ try: path_scheme = FileSystems.get_scheme(path) systems = [ fs for fs in FileSystem.get_all_subclasses() if fs.scheme() == path_scheme ] if len(systems) == 0: raise ValueError('Unable to get the Filesystem for path %s' % path) elif len(systems) == 1: return systems[0]() else: raise ValueError('Found more than one filesystem for path %s' % path) except ValueError: raise except Exception as e: raise BeamIOError('Unable to get the Filesystem', {path: e})
def match(self, patterns, limits=None): """Find all matching paths to the pattern provided. Args: patterns: list of string for the file path pattern to match against limits: list of maximum number of responses that need to be fetched Returns: list of ``MatchResult`` objects. Raises: ``BeamIOError`` if any of the pattern match operations fail """ if limits is None: limits = [None] * len(patterns) else: err_msg = "Patterns and limits should be equal in length" assert len(patterns) == len(limits), err_msg def _match(pattern, limit): """Find all matching paths to the pattern provided. """ files = glob.glob(pattern) metadata = [ FileMetadata(f, os.path.getsize(f)) for f in files[:limit] ] return MatchResult(pattern, metadata) exceptions = {} result = [] for pattern, limit in zip(patterns, limits): try: result.append(_match(pattern, limit)) except Exception as e: # pylint: disable=broad-except exceptions[pattern] = e if exceptions: raise BeamIOError("Match operation failed", exceptions) return result
def test_pre_finalize_error(self, delete_mock): temp_path = os.path.join(self._new_tempdir(), 'pre_finalize') sink = MyFileBasedSink( temp_path, file_name_suffix='.output', coder=coders.ToStringCoder()) init_token, [res1, res2] = self._common_init(sink) # no-op sink.pre_finalize(init_token, [res1, res2]) # Create finalized outputs from a previous run, which pre_finalize should # delete. shard1 = temp_path + '-00000-of-00002.output' shard2 = temp_path + '-00001-of-00002.output' with open(shard1, 'w') as f: f.write('foo') with open(shard2, 'w') as f: f.write('foo') error_str = 'mock rename error description' delete_mock.side_effect = BeamIOError( 'mock rename error', {shard2: error_str}) with self.assertRaisesRegexp(Exception, error_str): sink.pre_finalize(init_token, [res1, res2])
class TestUtilsTest(unittest.TestCase): def setUp(self): utils.patch_retry(self, utils) self.tmpdir = tempfile.mkdtemp() def test_delete_files_succeeds(self): f = tempfile.NamedTemporaryFile(dir=self.tmpdir, delete=False) assert FileSystems.exists(f.name) utils.delete_files([f.name]) assert not FileSystems.exists(f.name) @patch.object(FileSystems, 'delete', side_effect=BeamIOError('')) def test_delete_files_fails_with_io_error(self, mocked_delete): f = tempfile.NamedTemporaryFile(dir=self.tmpdir, delete=False) assert FileSystems.exists(f.name) with self.assertRaises(BeamIOError): utils.delete_files([f.name]) self.assertTrue(mocked_delete.called) self.assertEqual(mocked_delete.call_count, 4) def test_delete_files_fails_with_invalid_arg(self): with self.assertRaises(RuntimeError): utils.delete_files([])
def copy(self, source_file_names, destination_file_names): """Recursively copy the file tree from the source to the destination Args: source_file_names: list of source file objects that needs to be copied destination_file_names: list of destination of the new object Raises: ``BeamIOError`` if any of the copy operations fail """ err_msg = ("source_file_names and destination_file_names should " "be equal in length") assert len(source_file_names) == len(destination_file_names), err_msg def _copy_path(source, destination): """Recursively copy the file tree from the source to the destination """ if not destination.startswith(GCSFileSystem.GCS_PREFIX): raise ValueError('Destination %r must be GCS path.' % destination) # Use copy_tree if the path ends with / as it is a directory if source.endswith('/'): gcsio.GcsIO().copytree(source, destination) else: gcsio.GcsIO().copy(source, destination) exceptions = {} for source, destination in zip(source_file_names, destination_file_names): try: _copy_path(source, destination) except Exception as e: # pylint: disable=broad-except exceptions[(source, destination)] = e if exceptions: raise BeamIOError("Copy operation failed", exceptions)
def _list(self, dir_or_prefix): """List files in a location. Listing is non-recursive, for filesystems that support directories. Args: dir_or_prefix: (string) A directory or location prefix (for filesystems that don't have directories). Returns: Generator of ``FileMetadata`` objects. Raises: ``BeamIOError`` if listing fails, but not if no files were found. """ if not self.exists(dir_or_prefix): return try: for f in os.listdir(dir_or_prefix): f = self.join(dir_or_prefix, f) yield FileMetadata(f, os.path.getsize(f)) except Exception as e: # pylint: disable=broad-except raise BeamIOError("List operation failed", {dir_or_prefix: e})
def rename(self, source_file_names, destination_file_names): """Rename the files at the source list to the destination list. Source and destination lists should be of the same size. Args: source_file_names: List of file paths that need to be moved destination_file_names: List of destination_file_names for the files Raises: ``BeamIOError`` if any of the rename operations fail """ err_msg = ("source_file_names and destination_file_names should " "be equal in length") assert len(source_file_names) == len(destination_file_names), err_msg def _rename_file(source, destination): """Rename a single file object""" try: parent = os.path.dirname(destination) if not os.path.exists(parent): os.makedirs(parent) os.rename(source, destination) except OSError as err: raise IOError(err) exceptions = {} for source, destination in zip(source_file_names, destination_file_names): try: _rename_file(source, destination) except Exception as e: # pylint: disable=broad-except exceptions[(source, destination)] = e if exceptions: raise BeamIOError("Rename operation failed", exceptions)
def finalize_write(self, init_result, writer_results, unused_pre_finalize_results): writer_results = sorted(writer_results) num_shards = len(writer_results) src_files = [] dst_files = [] delete_files = [] chunk_size = FileSystems.get_chunk_size(self.file_path_prefix.get()) num_skipped = 0 for shard_num, shard in enumerate(writer_results): final_name = self._get_final_name(shard_num, num_shards) src = shard dst = final_name src_exists = FileSystems.exists(src) dst_exists = FileSystems.exists(dst) if not src_exists and not dst_exists: raise BeamIOError( 'src and dst files do not exist. src: %s, dst: %s' % (src, dst)) if not src_exists and dst_exists: logging.debug('src: %s -> dst: %s already renamed, skipping', src, dst) num_skipped += 1 continue if (src_exists and dst_exists and FileSystems.checksum(src) == FileSystems.checksum(dst)): logging.debug('src: %s == dst: %s, deleting src', src, dst) delete_files.append(src) continue src_files.append(src) dst_files.append(dst) num_skipped = len(delete_files) FileSystems.delete(delete_files) num_shards_to_finalize = len(src_files) min_threads = min(num_shards_to_finalize, FileBasedSink._MAX_RENAME_THREADS) num_threads = max(1, min_threads) source_file_batch = [ src_files[i:i + chunk_size] for i in range(0, len(src_files), chunk_size) ] destination_file_batch = [ dst_files[i:i + chunk_size] for i in range(0, len(dst_files), chunk_size) ] if num_shards_to_finalize: logging.info( 'Starting finalize_write threads with num_shards: %d (skipped: %d), ' 'batches: %d, num_threads: %d', num_shards_to_finalize, num_skipped, len(source_file_batch), num_threads) start_time = time.time() # Use a thread pool for renaming operations. def _rename_batch(batch): """_rename_batch executes batch rename operations.""" source_files, destination_files = batch exceptions = [] try: FileSystems.rename(source_files, destination_files) return exceptions except BeamIOError as exp: if exp.exception_details is None: raise for (src, dst), exception in exp.exception_details.iteritems(): if exception: logging.error( ('Exception in _rename_batch. src: %s, ' 'dst: %s, err: %s'), src, dst, exception) exceptions.append(exception) else: logging.debug('Rename successful: %s -> %s', src, dst) return exceptions exception_batches = util.run_using_threadpool( _rename_batch, zip(source_file_batch, destination_file_batch), num_threads) all_exceptions = [ e for exception_batch in exception_batches for e in exception_batch ] if all_exceptions: raise Exception( 'Encountered exceptions in finalize_write: %s' % all_exceptions) for final_name in dst_files: yield final_name logging.info('Renamed %d shards in %.2f seconds.', num_shards_to_finalize, time.time() - start_time) else: logging.warning( 'No shards found to finalize. num_shards: %d, skipped: %d', num_shards, num_skipped) try: FileSystems.delete([init_result]) except IOError: # May have already been removed. pass
def mkdirs(self, url): path = self._parse_url(url) if self._exists(path): raise BeamIOError('Path already exists: %s' % path) return self._mkdirs(path)
def size(self, url): path = self._parse_url(url) status = self._hdfs_client.status(path, strict=False) if status is None: raise BeamIOError('File not found: %s' % url) return status[_FILE_STATUS_LENGTH]