-
Notifications
You must be signed in to change notification settings - Fork 4.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-802] Python templates #2545
Changes from all commits
888b7bc
38a279e
52843be
152c7b5
46e98bd
675b809
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -14,6 +14,7 @@ | |
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
# | ||
|
||
"""File-based sources and sinks.""" | ||
|
||
from __future__ import absolute_import | ||
|
@@ -30,6 +31,9 @@ | |
from apache_beam.io.filesystem import CompressionTypes | ||
from apache_beam.io.filesystems_util import get_filesystem | ||
from apache_beam.transforms.display import DisplayDataItem | ||
from apache_beam.utils.value_provider import ValueProvider | ||
from apache_beam.utils.value_provider import StaticValueProvider | ||
from apache_beam.utils.value_provider import check_accessible | ||
|
||
DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' | ||
|
||
|
@@ -148,33 +152,39 @@ def __init__(self, | |
compression_type=CompressionTypes.AUTO): | ||
""" | ||
Raises: | ||
TypeError: if file path parameters are not a string or if compression_type | ||
is not member of CompressionTypes. | ||
TypeError: if file path parameters are not a string or ValueProvider, | ||
or if compression_type is not member of CompressionTypes. | ||
ValueError: if shard_name_template is not of expected format. | ||
""" | ||
if not isinstance(file_path_prefix, basestring): | ||
raise TypeError('file_path_prefix must be a string; got %r instead' % | ||
file_path_prefix) | ||
if not isinstance(file_name_suffix, basestring): | ||
raise TypeError('file_name_suffix must be a string; got %r instead' % | ||
file_name_suffix) | ||
if not isinstance(file_path_prefix, (basestring, ValueProvider)): | ||
raise TypeError('file_path_prefix must be a string or ValueProvider;' | ||
'got %r instead' % file_path_prefix) | ||
if not isinstance(file_name_suffix, (basestring, ValueProvider)): | ||
raise TypeError('file_name_suffix must be a string or ValueProvider;' | ||
'got %r instead' % file_name_suffix) | ||
|
||
if not CompressionTypes.is_valid_compression_type(compression_type): | ||
raise TypeError('compression_type must be CompressionType object but ' | ||
'was %s' % type(compression_type)) | ||
|
||
if shard_name_template is None: | ||
shard_name_template = DEFAULT_SHARD_NAME_TEMPLATE | ||
elif shard_name_template == '': | ||
num_shards = 1 | ||
if isinstance(file_path_prefix, basestring): | ||
file_path_prefix = StaticValueProvider(str, file_path_prefix) | ||
if isinstance(file_name_suffix, basestring): | ||
file_name_suffix = StaticValueProvider(str, file_name_suffix) | ||
self.file_path_prefix = file_path_prefix | ||
self.file_name_suffix = file_name_suffix | ||
self.num_shards = num_shards | ||
self.coder = coder | ||
self.shard_name_format = self._template_to_format(shard_name_template) | ||
self.compression_type = compression_type | ||
self.mime_type = mime_type | ||
self._file_system = get_filesystem(file_path_prefix) | ||
if file_path_prefix.is_accessible(): | ||
self._file_system = get_filesystem(file_path_prefix.get()) | ||
else: | ||
self._file_system = None | ||
|
||
def display_data(self): | ||
return {'shards': | ||
|
@@ -188,12 +198,15 @@ def display_data(self): | |
self.file_name_suffix), | ||
label='File Pattern')} | ||
|
||
@check_accessible(['file_path_prefix']) | ||
def open(self, temp_path): | ||
"""Opens ``temp_path``, returning an opaque file handle object. | ||
|
||
The returned file handle is passed to ``write_[encoded_]record`` and | ||
``close``. | ||
""" | ||
if self._file_system is None: | ||
self._file_system = get_filesystem(self.file_path_prefix.get()) | ||
return self._file_system.create(temp_path, self.mime_type, | ||
self.compression_type) | ||
|
||
|
@@ -220,22 +233,33 @@ def close(self, file_handle): | |
if file_handle is not None: | ||
file_handle.close() | ||
|
||
@check_accessible(['file_path_prefix', 'file_name_suffix']) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @sammcveety random question: Pros:
Cons:
The user can already just ignore the check if they want so I guess it is already sort of optional. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Java SDK does not have a concept similar to Using |
||
def initialize_write(self): | ||
tmp_dir = self.file_path_prefix + self.file_name_suffix + time.strftime( | ||
file_path_prefix = self.file_path_prefix.get() | ||
file_name_suffix = self.file_name_suffix.get() | ||
tmp_dir = file_path_prefix + file_name_suffix + time.strftime( | ||
'-temp-%Y-%m-%d_%H-%M-%S') | ||
if self._file_system is None: | ||
self._file_system = get_filesystem(file_path_prefix) | ||
self._file_system.mkdirs(tmp_dir) | ||
return tmp_dir | ||
|
||
@check_accessible(['file_path_prefix', 'file_name_suffix']) | ||
def open_writer(self, init_result, uid): | ||
# A proper suffix is needed for AUTO compression detection. | ||
# We also ensure there will be no collisions with uid and a | ||
# (possibly unsharded) file_path_prefix and a (possibly empty) | ||
# file_name_suffix. | ||
file_path_prefix = self.file_path_prefix.get() | ||
file_name_suffix = self.file_name_suffix.get() | ||
suffix = ( | ||
'.' + os.path.basename(self.file_path_prefix) + self.file_name_suffix) | ||
'.' + os.path.basename(file_path_prefix) + file_name_suffix) | ||
return FileSinkWriter(self, os.path.join(init_result, uid) + suffix) | ||
|
||
@check_accessible(['file_path_prefix', 'file_name_suffix']) | ||
def finalize_write(self, init_result, writer_results): | ||
file_path_prefix = self.file_path_prefix.get() | ||
file_name_suffix = self.file_name_suffix.get() | ||
writer_results = sorted(writer_results) | ||
num_shards = len(writer_results) | ||
min_threads = min(num_shards, FileSink._MAX_RENAME_THREADS) | ||
|
@@ -246,8 +270,8 @@ def finalize_write(self, init_result, writer_results): | |
chunk_size = self._file_system.CHUNK_SIZE | ||
for shard_num, shard in enumerate(writer_results): | ||
final_name = ''.join([ | ||
self.file_path_prefix, self.shard_name_format % dict( | ||
shard_num=shard_num, num_shards=num_shards), self.file_name_suffix | ||
file_path_prefix, self.shard_name_format % dict( | ||
shard_num=shard_num, num_shards=num_shards), file_name_suffix | ||
]) | ||
source_files.append(shard) | ||
destination_files.append(final_name) | ||
|
@@ -270,6 +294,8 @@ def _rename_batch(batch): | |
"""_rename_batch executes batch rename operations.""" | ||
source_files, destination_files = batch | ||
exceptions = [] | ||
if self._file_system is None: | ||
self._file_system = get_filesystem(file_path_prefix) | ||
try: | ||
self._file_system.rename(source_files, destination_files) | ||
return exceptions | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Update json_values_tests to have this case ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
https://issues.apache.org/jira/browse/BEAM-1998 for @mariapython