diff --git a/sdks/python/apache_beam/__init__.py b/sdks/python/apache_beam/__init__.py index 791ebb7a342e..eec352d60b1c 100644 --- a/sdks/python/apache_beam/__init__.py +++ b/sdks/python/apache_beam/__init__.py @@ -77,8 +77,9 @@ if not (sys.version_info[0] == 2 and sys.version_info[1] == 7): raise RuntimeError( - 'The Apache Beam SDK for Python is supported only on Python 2.7. ' - 'It is not supported on Python ['+ str(sys.version_info) + '].') + 'The Apache Beam SDK for Python is supported only on Python 2.7.' + 'It is not supported on Python {0}.' + .format(sys.version)) # pylint: disable=wrong-import-position import apache_beam.internal.pickler diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 2670250c36b6..230cefb8d44c 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -26,28 +26,37 @@ For internal use only; no backwards-compatibility guarantees. """ -from types import NoneType +from __future__ import absolute_import, division + +import sys +from builtins import chr, object, range + +from past.utils import old_div from apache_beam.coders import observable -from apache_beam.utils.timestamp import Timestamp -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP from apache_beam.utils import windowed_value +from apache_beam.utils.timestamp import MAX_TIMESTAMP, MIN_TIMESTAMP, Timestamp + +if sys.version_info[0] >= 3: + basestring = str + long = int + unicode = str + # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: - from stream import InputStream as create_InputStream - from stream import OutputStream as create_OutputStream - from stream import ByteCountingOutputStream - from stream import get_varint_size + from .stream import InputStream as create_InputStream + from .stream import OutputStream as create_OutputStream + from .stream import ByteCountingOutputStream + from .stream import get_varint_size globals()['create_InputStream'] = create_InputStream globals()['create_OutputStream'] = create_OutputStream globals()['ByteCountingOutputStream'] = ByteCountingOutputStream except ImportError: - from slow_stream import InputStream as create_InputStream - from slow_stream import OutputStream as create_OutputStream - from slow_stream import ByteCountingOutputStream - from slow_stream import get_varint_size + from .slow_stream import InputStream as create_InputStream + from .slow_stream import OutputStream as create_OutputStream + from .slow_stream import ByteCountingOutputStream + from .slow_stream import get_varint_size # pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports @@ -154,7 +163,8 @@ def encode_to_stream(self, value, stream, nested): return stream.write(self._encoder(value), nested) def decode_from_stream(self, stream, nested): - return self._decoder(stream.read_all(nested)) + read_from_stream = stream.read_all(nested) + return self._decoder(read_from_stream) def encode(self, value): return self._encoder(value) @@ -182,7 +192,7 @@ def __init__(self, coder, step_label): self._step_label = step_label def _check_safe(self, value): - if isinstance(value, (str, unicode, long, int, float)): + if isinstance(value, (str, basestring, bytes, int, long, float)): pass elif value is None: pass @@ -262,7 +272,7 @@ def get_estimated_size_and_observables(self, value, nested=False): def encode_to_stream(self, value, stream, nested): t = type(value) - if t is NoneType: + if value is None: stream.write_byte(NONE_TYPE) elif t is int: stream.write_byte(INT_TYPE) @@ -270,10 +280,10 @@ def encode_to_stream(self, value, stream, nested): elif t is float: stream.write_byte(FLOAT_TYPE) stream.write_bigendian_double(value) - elif t is str: + elif t is bytes: stream.write_byte(STR_TYPE) stream.write(value, nested) - elif t is unicode: + elif t is unicode or t is basestring: unicode_value = value # for typing stream.write_byte(UNICODE_TYPE) stream.write(unicode_value.encode('utf-8'), nested) @@ -287,7 +297,7 @@ def encode_to_stream(self, value, stream, nested): dict_value = value # for typing stream.write_byte(DICT_TYPE) stream.write_var_int64(len(dict_value)) - for k, v in dict_value.iteritems(): + for k, v in dict_value.items(): self.encode_to_stream(k, stream, True) self.encode_to_stream(v, stream, True) elif t is bool: @@ -306,14 +316,17 @@ def decode_from_stream(self, stream, nested): elif t == FLOAT_TYPE: return stream.read_bigendian_double() elif t == STR_TYPE: - return stream.read_all(nested) + if sys.version_info[0] < 3: + return str(stream.read_all(nested)) + else: + return stream.read_all(nested) elif t == UNICODE_TYPE: return stream.read_all(nested).decode('utf-8') elif t == LIST_TYPE or t == TUPLE_TYPE or t == SET_TYPE: vlen = stream.read_var_int64() vlist = [self.decode_from_stream(stream, True) for _ in range(vlen)] if t == LIST_TYPE: - return vlist + return list(vlist) elif t == TUPLE_TYPE: return tuple(vlist) return set(vlist) @@ -333,7 +346,7 @@ def decode_from_stream(self, stream, nested): class BytesCoderImpl(CoderImpl): """For internal use only; no backwards-compatibility guarantees. - A coder for bytes/str objects.""" + A coder for bytes/str objects. In Python3 this will return bytes not strs.""" def encode_to_stream(self, value, out, nested): out.write(value, nested) @@ -342,7 +355,11 @@ def decode_from_stream(self, in_stream, nested): return in_stream.read_all(nested) def encode(self, value): - assert isinstance(value, bytes), (value, type(value)) + assert isinstance(value, (bytes, str)) + if isinstance(value, bytes): + return value + elif isinstance(value, str): + return value.encode('latin-1') return value def decode(self, encoded): @@ -377,8 +394,9 @@ def _from_normal_time(self, value): def encode_to_stream(self, value, out, nested): span_micros = value.end.micros - value.start.micros - out.write_bigendian_uint64(self._from_normal_time(value.end.micros / 1000)) - out.write_var_int64(span_micros / 1000) + out.write_bigendian_uint64(self._from_normal_time( + old_div(value.end.micros, 1000))) + out.write_var_int64(old_div(span_micros, 1000)) def decode_from_stream(self, in_, nested): end_millis = self._to_normal_time(in_.read_bigendian_uint64()) @@ -392,7 +410,7 @@ def estimate_size(self, value, nested=False): # An IntervalWindow is context-insensitive, with a timestamp (8 bytes) # and a varint timespam. span = value.end.micros - value.start.micros - return 8 + get_varint_size(span / 1000) + return 8 + get_varint_size(old_div(span, 1000)) class TimestampCoderImpl(StreamCoderImpl): @@ -689,7 +707,7 @@ def encode_to_stream(self, value, out, nested): # TODO(BEAM-1524): Clean this up once we have a BEAM wide consensus on # precision of timestamps. self._from_normal_time( - restore_sign * (abs(wv.timestamp_micros) / 1000))) + restore_sign * (old_div(abs(wv.timestamp_micros), 1000)))) self._windows_coder.encode_to_stream(wv.windows, out, True) # Default PaneInfo encoded byte representing NO_FIRING. # TODO(BEAM-1522): Remove the hard coding here once PaneInfo is supported. @@ -704,9 +722,9 @@ def decode_from_stream(self, in_stream, nested): # were indeed MIN/MAX timestamps. # TODO(BEAM-1524): Clean this up once we have a BEAM wide consensus on # precision of timestamps. - if timestamp == -(abs(MIN_TIMESTAMP.micros) / 1000): + if timestamp == -(old_div(abs(MIN_TIMESTAMP.micros), 1000)): timestamp = MIN_TIMESTAMP.micros - elif timestamp == (MAX_TIMESTAMP.micros / 1000): + elif timestamp == (old_div(MAX_TIMESTAMP.micros, 1000)): timestamp = MAX_TIMESTAMP.micros else: timestamp *= 1000 diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index e204369b3103..461598f852fb 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -19,21 +19,32 @@ Only those coders listed in __all__ are part of the public API of this module. """ +from __future__ import absolute_import import base64 -import cPickle as pickle +import sys +from builtins import object + import google.protobuf +from future import standard_library from apache_beam.coders import coder_impl from apache_beam.portability.api import beam_runner_api_pb2 -from apache_beam.utils import urns -from apache_beam.utils import proto_utils +from apache_beam.utils import proto_utils, urns + +standard_library.install_aliases() + +if sys.version_info[0] >= 3: + import pickle as pickle +else: + import cPickle as pickle + # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: - from stream import get_varint_size + from .stream import get_varint_size except ImportError: - from slow_stream import get_varint_size + from .slow_stream import get_varint_size # pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports @@ -52,7 +63,7 @@ __all__ = ['Coder', 'BytesCoder', 'DillCoder', 'FastPrimitivesCoder', 'FloatCoder', 'IterableCoder', 'PickleCoder', 'ProtoCoder', 'SingletonCoder', - 'StrUtf8Coder', 'TimestampCoder', 'TupleCoder', + 'StrUtf8Coder', 'StrUtf8StrCoder', 'TimestampCoder', 'TupleCoder', 'TupleSequenceCoder', 'VarIntCoder', 'WindowedValueCoder'] @@ -291,11 +302,26 @@ def is_deterministic(self): return True +class StrUtf8StrCoder(Coder): + """A coder used for reading and writing strings as UTF-8. + Used for Python 2 to force into string rather than unicode on decode.""" + + def encode(self, value): + return value.encode('utf-8') + + def decode(self, value): + return str(value.decode('utf-8')) + + def is_deterministic(self): + return True + + class ToStringCoder(Coder): """A default string coder used if no sink coder is specified.""" def encode(self, value): - if isinstance(value, unicode): + # TODO(holden before merge): Do we just want to send everything to UTF-8? + if isinstance(value, str): return value.encode('utf-8') elif isinstance(value, str): return value diff --git a/sdks/python/apache_beam/coders/coders_test.py b/sdks/python/apache_beam/coders/coders_test.py index c89e81028b8a..61385eee660b 100644 --- a/sdks/python/apache_beam/coders/coders_test.py +++ b/sdks/python/apache_beam/coders/coders_test.py @@ -19,9 +19,10 @@ import base64 import logging import unittest +from builtins import object -from apache_beam.coders import coders from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message +from apache_beam.coders import coders from apache_beam.coders.typecoders import registry as coders_registry diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index 8b0353d922e7..fdde00c6cb44 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -16,23 +16,25 @@ # """Tests common to all coder implementations.""" +from __future__ import absolute_import, print_function import logging import math +import sys import unittest +from builtins import range import dill -from apache_beam.transforms.window import GlobalWindow -from apache_beam.utils.timestamp import MIN_TIMESTAMP -import observable +from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message +from apache_beam.coders import coders from apache_beam.runners import pipeline_context from apache_beam.transforms import window -from apache_beam.utils import timestamp -from apache_beam.utils import windowed_value +from apache_beam.transforms.window import GlobalWindow +from apache_beam.utils import timestamp, windowed_value +from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.coders import coders -from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message +from . import observable # Defined out of line for picklability. @@ -58,13 +60,14 @@ def setUpClass(cls): @classmethod def tearDownClass(cls): standard = set(c - for c in coders.__dict__.values() + for c in list(coders.__dict__.values()) if isinstance(c, type) and issubclass(c, coders.Coder) and 'Base' not in c.__name__) standard -= set([coders.Coder, coders.FastCoder, coders.ProtoCoder, - coders.ToStringCoder]) + coders.ToStringCoder, + coders.StrUtf8StrCoder]) assert not standard - cls.seen, standard - cls.seen assert not standard - cls.seen_nested, standard - cls.seen_nested @@ -80,10 +83,29 @@ def _observe_nested(cls, coder): cls.seen_nested.add(type(c)) cls._observe_nested(c) + def assertItemsEqual(self, a, b): + if sys.version_info[0] >= 3: + self.assertCountEqual(a, b) + else: + self.assertEqual(sorted(a), sorted(b)) + def check_coder(self, coder, *values): self._observe(coder) for v in values: - self.assertEqual(v, coder.decode(coder.encode(v))) + try: + encoded = coder.encode(v) + except Exception: + print("Error encoding {0}".format(str(v))) + raise + try: + decoded = coder.decode(encoded) + except Exception: + print("Error decoding {0}".format(str(v))) + raise e + if isinstance(v, str) and isinstance(decoded, bytes): + self.assertEqual(v, decoded.decode("utf-8")) + else: + self.assertEqual(v, decoded) self.assertEqual(coder.estimate_size(v), len(coder.encode(v))) self.assertEqual(coder.estimate_size(v), @@ -94,7 +116,11 @@ def check_coder(self, coder, *values): context = pipeline_context.PipelineContext() copy2 = coders.Coder.from_runner_api(coder.to_runner_api(context), context) for v in values: - self.assertEqual(v, copy1.decode(copy2.encode(v))) + decoded = copy1.decode(copy2.encode(v)) + if isinstance(v, str) and not isinstance(decoded, str): + self.assertEqual(v, decoded.decode("utf-8")) + else: + self.assertEqual(v, decoded) if coder.is_deterministic(): self.assertEqual(copy1.encode(v), copy2.encode(v)) @@ -102,7 +128,7 @@ def test_custom_coder(self): self.check_coder(CustomCoder(), 1, -10, 5) self.check_coder(coders.TupleCoder((CustomCoder(), coders.BytesCoder())), - (1, 'a'), (-10, 'b'), (5, 'c')) + (1, b'a'), (-10, b'b'), (5, b'c')) def test_pickle_coder(self): self.check_coder(coders.PickleCoder(), 'a', 1, 1.5, (1, 2, 3)) @@ -117,11 +143,11 @@ def test_deterministic_coder(self): self.check_coder(deterministic_coder, [1, dict()]) self.check_coder(coders.TupleCoder((deterministic_coder, coder)), - (1, dict()), ('a', [dict()])) + (1, dict()), (b'a', [dict()])) def test_dill_coder(self): - cell_value = (lambda x: lambda: x)(0).func_closure[0] - self.check_coder(coders.DillCoder(), 'a', 1, cell_value) + cell_value = (lambda x: lambda: x)(0).__closure__[0] + self.check_coder(coders.DillCoder(), b'a', 1, cell_value) self.check_coder( coders.TupleCoder((coders.VarIntCoder(), coders.DillCoder())), (1, cell_value)) @@ -131,20 +157,20 @@ def test_fast_primitives_coder(self): self.check_coder(coder, None, 1, -1, 1.5, 'str\0str', u'unicode\0\u0101') self.check_coder(coder, (), (1, 2, 3)) self.check_coder(coder, [], [1, 2, 3]) - self.check_coder(coder, dict(), {'a': 'b'}, {0: dict(), 1: len}) - self.check_coder(coder, set(), {'a', 'b'}) + self.check_coder(coder, dict(), {b'a': b'b'}, {0: dict(), 1: len}) + self.check_coder(coder, set(), {b'a', b'b'}) self.check_coder(coder, True, False) self.check_coder(coder, len) self.check_coder(coders.TupleCoder((coder,)), ('a',), (1,)) def test_bytes_coder(self): - self.check_coder(coders.BytesCoder(), 'a', '\0', 'z' * 1000) + self.check_coder(coders.BytesCoder(), b'a', b'\0', b'z' * 1000) def test_varint_coder(self): # Small ints. - self.check_coder(coders.VarIntCoder(), *range(-10, 10)) + self.check_coder(coders.VarIntCoder(), *list(range(-10, 10))) # Multi-byte encoding starts at 128 - self.check_coder(coders.VarIntCoder(), *range(120, 140)) + self.check_coder(coders.VarIntCoder(), *list(range(120, 140))) # Large values MAX_64_BIT_INT = 0x7fffffffffffffff self.check_coder(coders.VarIntCoder(), @@ -189,7 +215,7 @@ def test_timestamp_coder(self): timestamp.Timestamp(micros=1234567890123456789)) self.check_coder( coders.TupleCoder((coders.TimestampCoder(), coders.BytesCoder())), - (timestamp.Timestamp.of(27), 'abc')) + (timestamp.Timestamp.of(27), b'abc')) def test_tuple_coder(self): kv_coder = coders.TupleCoder((coders.VarIntCoder(), coders.BytesCoder())) @@ -205,14 +231,14 @@ def test_tuple_coder(self): kv_coder.as_cloud_object()) # Test binary representation self.assertEqual( - '\x04abc', - kv_coder.encode((4, 'abc'))) + b'\x04abc', + kv_coder.encode((4, b'abc'))) # Test unnested self.check_coder( kv_coder, - (1, 'a'), - (-2, 'a' * 100), - (300, 'abc\0' * 5)) + (1, b'a'), + (-2, b'a' * 100), + (300, b'abc\0' * 5)) # Test nested self.check_coder( coders.TupleCoder( @@ -232,6 +258,9 @@ def test_tuple_sequence_coder(self): def test_base64_pickle_coder(self): self.check_coder(coders.Base64PickleCoder(), 'a', 1, 1.5, (1, 2, 3)) + def test_basic_str_coder(self): + self.check_coder(coders.StrUtf8StrCoder(), 'a', 'b', 'ee') + def test_utf8_coder(self): self.check_coder(coders.StrUtf8Coder(), 'a', u'ab\u00FF', u'\u0101\0') @@ -289,12 +318,12 @@ def test_windowed_value_coder(self): }, coder.as_cloud_object()) # Test binary representation - self.assertEqual('\x7f\xdf;dZ\x1c\xac\t\x00\x00\x00\x01\x0f\x01', + self.assertEqual(b'\x7f\xdf;dZ\x1c\xac\t\x00\x00\x00\x01\x0f\x01', coder.encode(window.GlobalWindows.windowed_value(1))) # Test decoding large timestamp self.assertEqual( - coder.decode('\x7f\xdf;dZ\x1c\xac\x08\x00\x00\x00\x01\x0f\x00'), + coder.decode(b'\x7f\xdf;dZ\x1c\xac\x08\x00\x00\x00\x01\x0f\x00'), windowed_value.create(0, MIN_TIMESTAMP.micros, (GlobalWindow(),))) # Test unnested @@ -331,7 +360,7 @@ def test_proto_coder(self): proto_coder = coders.ProtoCoder(ma.__class__) self.check_coder(proto_coder, ma) self.check_coder(coders.TupleCoder((proto_coder, coders.BytesCoder())), - (ma, 'a'), (mb, 'b')) + (ma, b'a'), (mb, b'b')) def test_global_window_coder(self): coder = coders.GlobalWindowCoder() @@ -358,16 +387,16 @@ def test_length_prefix_coder(self): }, coder.as_cloud_object()) # Test binary representation - self.assertEqual('\x00', coder.encode('')) - self.assertEqual('\x01a', coder.encode('a')) - self.assertEqual('\x02bc', coder.encode('bc')) - self.assertEqual('\xff\x7f' + 'z' * 16383, coder.encode('z' * 16383)) + self.assertEqual(b'\x00', coder.encode(b'')) + self.assertEqual(b'\x01a', coder.encode(b'a')) + self.assertEqual(b'\x02bc', coder.encode(b'bc')) + self.assertEqual(b'\xff\x7f' + b'z' * 16383, coder.encode(b'z' * 16383)) # Test unnested - self.check_coder(coder, '', 'a', 'bc', 'def') + self.check_coder(coder, b'', b'a', b'bc', b'def') # Test nested self.check_coder(coders.TupleCoder((coder, coder)), - ('', 'a'), - ('bc', 'def')) + (b'', b'a'), + (b'bc', b'def')) def test_nested_observables(self): class FakeObservableIterator(observable.ObservableMixin): diff --git a/sdks/python/apache_beam/coders/fast_coders_test.py b/sdks/python/apache_beam/coders/fast_coders_test.py index a13334a2c26f..d060117a147e 100644 --- a/sdks/python/apache_beam/coders/fast_coders_test.py +++ b/sdks/python/apache_beam/coders/fast_coders_test.py @@ -20,8 +20,6 @@ import logging import unittest - -# Run all the standard coder test cases. from apache_beam.coders.coders_test_common import * diff --git a/sdks/python/apache_beam/coders/observable.py b/sdks/python/apache_beam/coders/observable.py index fc952cf4e559..5977624883f0 100644 --- a/sdks/python/apache_beam/coders/observable.py +++ b/sdks/python/apache_beam/coders/observable.py @@ -22,6 +22,9 @@ """ +from builtins import object + + class ObservableMixin(object): """For internal use only; no backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/coders/observable_test.py b/sdks/python/apache_beam/coders/observable_test.py index eaf1aec9e600..09ca3041c298 100644 --- a/sdks/python/apache_beam/coders/observable_test.py +++ b/sdks/python/apache_beam/coders/observable_test.py @@ -20,7 +20,6 @@ import logging import unittest - from apache_beam.coders import observable diff --git a/sdks/python/apache_beam/coders/proto2_coder_test_messages_pb2.py b/sdks/python/apache_beam/coders/proto2_coder_test_messages_pb2.py index 16b1b4d6b4b3..433d33f98841 100644 --- a/sdks/python/apache_beam/coders/proto2_coder_test_messages_pb2.py +++ b/sdks/python/apache_beam/coders/proto2_coder_test_messages_pb2.py @@ -19,19 +19,19 @@ # source: sdks/java/core/src/main/proto/proto2_coder_test_messages.proto import sys -_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) + from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database from google.protobuf import descriptor_pb2 + +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() - - DESCRIPTOR = _descriptor.FileDescriptor( name='apache_beam/coders/proto2_coder_test_messages.proto', package='proto2_coder_test_messages', diff --git a/sdks/python/apache_beam/coders/slow_coders_test.py b/sdks/python/apache_beam/coders/slow_coders_test.py index 97aa39ca094f..7032e081658a 100644 --- a/sdks/python/apache_beam/coders/slow_coders_test.py +++ b/sdks/python/apache_beam/coders/slow_coders_test.py @@ -20,8 +20,6 @@ import logging import unittest - -# Run all the standard coder test cases. from apache_beam.coders.coders_test_common import * diff --git a/sdks/python/apache_beam/coders/slow_stream.py b/sdks/python/apache_beam/coders/slow_stream.py index 1ab55d90f98d..b7f3cd5fc50a 100644 --- a/sdks/python/apache_beam/coders/slow_stream.py +++ b/sdks/python/apache_beam/coders/slow_stream.py @@ -21,6 +21,11 @@ """ import struct +import sys +from builtins import chr, object + +if sys.version_info[0] >= 3: + basestring = str class OutputStream(object): @@ -32,13 +37,16 @@ def __init__(self): self.data = [] def write(self, b, nested=False): - assert isinstance(b, str) + assert isinstance(b, (bytes, basestring)) if nested: self.write_var_int64(len(b)) - self.data.append(b) + if isinstance(b, bytes): + self.data.append(b) + else: + self.data.append(b.encode("latin-1")) def write_byte(self, val): - self.data.append(chr(val)) + self.write(chr(val)) def write_var_int64(self, v): if v < 0: @@ -67,7 +75,7 @@ def write_bigendian_double(self, v): self.write(struct.pack('>d', v)) def get(self): - return ''.join(self.data) + return b''.join(self.data) def size(self): return len(self.data) @@ -123,7 +131,8 @@ def read_all(self, nested): def read_byte(self): self.pos += 1 - return ord(self.data[self.pos - 1]) + elem = self.data[self.pos - 1:self.pos] + return ord(elem) def read_var_int64(self): shift = 0 @@ -135,7 +144,7 @@ def read_var_int64(self): bits = byte & 0x7F if shift >= 64 or (shift >= 63 and bits > 1): - raise RuntimeError('VarLong too long.') + raise RuntimeError('VarLong of size ' + str(shift) + ' too long.') result |= bits << shift shift += 7 if not byte & 0x80: diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py index 5f9845564477..39e03463aa14 100644 --- a/sdks/python/apache_beam/coders/standard_coders_test.py +++ b/sdks/python/apache_beam/coders/standard_coders_test.py @@ -17,21 +17,22 @@ """Unit tests for coders that must be consistent across all Beam SDKs. """ +from __future__ import print_function import json import logging import os.path import sys import unittest +from builtins import map import yaml -from apache_beam.coders import coders -from apache_beam.coders import coder_impl +from apache_beam.coders import coder_impl, coders +from apache_beam.transforms import window +from apache_beam.transforms.window import IntervalWindow from apache_beam.utils import windowed_value from apache_beam.utils.timestamp import Timestamp -from apache_beam.transforms.window import IntervalWindow -from apache_beam.transforms import window STANDARD_CODERS_YAML = os.path.join( os.path.dirname(__file__), '..', 'testing', 'data', 'standard_coders.yaml') @@ -73,7 +74,7 @@ class StandardCodersTest(unittest.TestCase): lambda x: IntervalWindow( start=Timestamp(micros=(x['end'] - x['span']) * 1000), end=Timestamp(micros=x['end'] * 1000)), - 'urn:beam:coders:stream:0.1': lambda x, parser: map(parser, x), + 'urn:beam:coders:stream:0.1': lambda x, parser: list(map(parser, x)), 'urn:beam:coders:global_window:0.1': lambda x: window.GlobalWindow(), 'urn:beam:coders:windowed_value:0.1': lambda x, value_parser, window_parser: windowed_value.create( @@ -91,7 +92,7 @@ def _run_standard_coder(self, name, spec): parse_value = self.json_value_parser(spec['coder']) nested_list = [spec['nested']] if 'nested' in spec else [True, False] for nested in nested_list: - for expected_encoded, json_value in spec['examples'].items(): + for expected_encoded, json_value in list(spec['examples'].items()): value = parse_value(json_value) expected_encoded = expected_encoded.encode('latin1') if not spec['coder'].get('non_deterministic', False): @@ -125,14 +126,14 @@ def json_value_parser(self, coder_spec): @classmethod def tearDownClass(cls): if cls.fix and cls.to_fix: - print "FIXING", len(cls.to_fix), "TESTS" + print("FIXING", len(cls.to_fix), "TESTS") doc_sep = '\n---\n' docs = open(STANDARD_CODERS_YAML).read().split(doc_sep) def quote(s): return json.dumps(s.decode('latin1')).replace(r'\u0000', r'\0') for (doc_ix, expected_encoded), actual_encoded in cls.to_fix.items(): - print quote(expected_encoded), "->", quote(actual_encoded) + print(quote(expected_encoded), "->", quote(actual_encoded)) docs[doc_ix] = docs[doc_ix].replace( quote(expected_encoded) + ':', quote(actual_encoded) + ':') open(STANDARD_CODERS_YAML, 'w').write(doc_sep.join(docs)) diff --git a/sdks/python/apache_beam/coders/stream_test.py b/sdks/python/apache_beam/coders/stream_test.py index e6108b68aee9..b2a58a16700c 100644 --- a/sdks/python/apache_beam/coders/stream_test.py +++ b/sdks/python/apache_beam/coders/stream_test.py @@ -16,11 +16,14 @@ # """Tests for the stream implementations.""" +from __future__ import division import logging import math import unittest +from builtins import range +from past.utils import old_div from apache_beam.coders import slow_stream @@ -60,7 +63,7 @@ def test_read_write_byte(self): self.assertEquals(0xFF, in_s.read_byte()) def test_read_write_large(self): - values = range(4 * 1024) + values = list(range(4 * 1024)) out_s = self.OutputStream() for v in values: out_s.write_bigendian_int64(v) @@ -77,7 +80,7 @@ def run_read_write_var_int64(self, values): self.assertEquals(v, in_s.read_var_int64()) def test_small_var_int64(self): - self.run_read_write_var_int64(range(-10, 30)) + self.run_read_write_var_int64(list(range(-10, 30))) def test_medium_var_int64(self): base = -1.7 @@ -89,7 +92,7 @@ def test_large_var_int64(self): self.run_read_write_var_int64([0, 2**63 - 1, -2**63, 2**63 - 3]) def test_read_write_double(self): - values = 0, 1, -1, 1e100, 1.0/3, math.pi, float('inf') + values = 0, 1, -1, 1e100, old_div(1.0, 3), math.pi, float('inf') out_s = self.OutputStream() for v in values: out_s.write_bigendian_double(v) diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py index 3894bb588172..1be98f341159 100644 --- a/sdks/python/apache_beam/coders/typecoders.py +++ b/sdks/python/apache_beam/coders/typecoders.py @@ -65,12 +65,13 @@ def MakeXyzs(v): """ import logging +import sys import warnings +from builtins import object from apache_beam.coders import coders from apache_beam.typehints import typehints - __all__ = ['registry'] @@ -88,7 +89,11 @@ def register_standard_coders(self, fallback_coder): self._register_coder_internal(float, coders.FloatCoder) self._register_coder_internal(str, coders.BytesCoder) self._register_coder_internal(bytes, coders.BytesCoder) - self._register_coder_internal(unicode, coders.StrUtf8Coder) + if sys.version_info[0] >= 3: + self._register_coder_internal(str, coders.StrUtf8Coder) + else: + self._register_coder_internal(str, coders.StrUtf8StrCoder) + self._register_coder_internal(unicode, coders.StrUtf8Coder) self._register_coder_internal(typehints.TupleConstraint, coders.TupleCoder) # Default fallback coders applied in that order until the first matching # coder found. diff --git a/sdks/python/apache_beam/coders/typecoders_test.py b/sdks/python/apache_beam/coders/typecoders_test.py index 2b6aa7a51298..0ce14f17e3df 100644 --- a/sdks/python/apache_beam/coders/typecoders_test.py +++ b/sdks/python/apache_beam/coders/typecoders_test.py @@ -18,9 +18,9 @@ """Unit tests for the typecoders module.""" import unittest +from builtins import object -from apache_beam.coders import coders -from apache_beam.coders import typecoders +from apache_beam.coders import coders, typecoders from apache_beam.internal import pickler from apache_beam.typehints import typehints diff --git a/sdks/python/apache_beam/examples/complete/autocomplete.py b/sdks/python/apache_beam/examples/complete/autocomplete.py index ab3397cfe335..271a84ee7e61 100644 --- a/sdks/python/apache_beam/examples/complete/autocomplete.py +++ b/sdks/python/apache_beam/examples/complete/autocomplete.py @@ -22,12 +22,11 @@ import argparse import logging import re +from builtins import range import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions def run(argv=None): @@ -44,14 +43,17 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True + + def format_result(prefix_candidates): + return '%s: %s' % (prefix_candidates[0], prefix_candidates[1]) + with beam.Pipeline(options=pipeline_options) as p: (p # pylint: disable=expression-not-assigned | 'read' >> ReadFromText(known_args.input) | 'split' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) | 'TopPerPrefix' >> TopPerPrefix(5) - | 'format' >> beam.Map( - lambda (prefix, candidates): '%s: %s' % (prefix, candidates)) + | 'format' >> beam.Map(format_result) | 'write' >> WriteToText(known_args.output)) diff --git a/sdks/python/apache_beam/examples/complete/autocomplete_test.py b/sdks/python/apache_beam/examples/complete/autocomplete_test.py index e2c84d68d3d4..0a1eeb330486 100644 --- a/sdks/python/apache_beam/examples/complete/autocomplete_test.py +++ b/sdks/python/apache_beam/examples/complete/autocomplete_test.py @@ -22,8 +22,7 @@ import apache_beam as beam from apache_beam.examples.complete import autocomplete from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class AutocompleteTest(unittest.TestCase): @@ -35,7 +34,7 @@ def test_top_prefixes(self): words = p | beam.Create(self.WORDS) result = words | autocomplete.TopPerPrefix(5) # values must be hashable for now - result = result | beam.Map(lambda (k, vs): (k, tuple(vs))) + result = result | beam.Map(lambda k_vs: (k_vs[0], tuple(k_vs[1]))) assert_that(result, equal_to( [ ('t', ((3, 'to'), (2, 'this'), (1, 'that'))), diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi.py b/sdks/python/apache_beam/examples/complete/estimate_pi.py index 7e3c4cd35a27..d7b5895e7be8 100644 --- a/sdks/python/apache_beam/examples/complete/estimate_pi.py +++ b/sdks/python/apache_beam/examples/complete/estimate_pi.py @@ -30,15 +30,12 @@ import json import logging import random - +from builtins import object, range import apache_beam as beam from apache_beam.io import WriteToText -from apache_beam.typehints import Any -from apache_beam.typehints import Iterable -from apache_beam.typehints import Tuple -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions +from apache_beam.typehints import Any, Iterable, Tuple @beam.typehints.with_output_types(Tuple[int, int, int]) @@ -56,7 +53,7 @@ def run_trials(runs): has same type for inputs and outputs (a requirement for combiner functions). """ inside_runs = 0 - for _ in xrange(runs): + for _ in range(runs): x = random.uniform(0, 1) y = random.uniform(0, 1) inside_runs += 1 if x * x + y * y <= 1.0 else 0 diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py index f1cbb0a24d56..94399da32499 100644 --- a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py +++ b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py @@ -22,8 +22,7 @@ from apache_beam.examples.complete import estimate_pi from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import BeamAssertException +from apache_beam.testing.util import BeamAssertException, assert_that def in_between(lower, upper): diff --git a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py index 9dd8b05a5c5c..e6aeddb0ec07 100644 --- a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py +++ b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py @@ -64,8 +64,7 @@ --temp_location gs://$BUCKET/user_score/temp """ -from __future__ import absolute_import -from __future__ import print_function +from __future__ import absolute_import, division, print_function import argparse import csv @@ -75,10 +74,10 @@ from datetime import datetime import apache_beam as beam -from apache_beam.metrics.metric import Metrics -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.metrics import Metrics +from apache_beam.options.pipeline_options import (GoogleCloudOptions, + PipelineOptions, + SetupOptions) def str2timestamp(s, fmt='%Y-%m-%d-%H-%M'): diff --git a/sdks/python/apache_beam/examples/complete/game/hourly_team_score_test.py b/sdks/python/apache_beam/examples/complete/game/hourly_team_score_test.py index 9c30127d7416..39fa72cb46d4 100644 --- a/sdks/python/apache_beam/examples/complete/game/hourly_team_score_test.py +++ b/sdks/python/apache_beam/examples/complete/game/hourly_team_score_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.complete.game import hourly_team_score from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class HourlyTeamScoreTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/complete/game/user_score_test.py b/sdks/python/apache_beam/examples/complete/game/user_score_test.py index 59903d985460..e2880213bf3c 100644 --- a/sdks/python/apache_beam/examples/complete/game/user_score_test.py +++ b/sdks/python/apache_beam/examples/complete/game/user_score_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.complete.game import user_score from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class UserScoreTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py index 61e3fd1a8d0c..1f69e40e566d 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py @@ -23,6 +23,7 @@ from __future__ import absolute_import import argparse +from builtins import range import apache_beam as beam from apache_beam.io import WriteToText @@ -37,7 +38,7 @@ def get_julia_set_point_color(element, c, n, max_iterations): """Given an pixel, convert it into a point in our julia set.""" x, y = element z = from_pixel(x, y, n) - for i in xrange(max_iterations): + for i in range(max_iterations): if z.real * z.real + z.imag * z.imag > 2.0: break z = z * z + c @@ -104,14 +105,18 @@ def run(argv=None): # pylint: disable=missing-docstring coordinates = generate_julia_set_colors(p, complex(-.62772, .42193), n, 100) + def x_coord_key(x_y_i): + return (x_y_i[0], (x_y_i[0], x_y_i[1], x_y_i[2])) + + def format_result(k_coords): + return ' '.join('(%s, %s, %s)' % c for c in k_coords[1]) # Group each coordinate triplet by its x value, then write the coordinates # to the output file with an x-coordinate grouping per line. # pylint: disable=expression-not-assigned (coordinates - | 'x coord key' >> beam.Map(lambda (x, y, i): (x, (x, y, i))) + | 'x coord key' >> beam.Map(x_coord_key) | 'x coord' >> beam.GroupByKey() - | 'format' >> beam.Map( - lambda (k, coords): ' '.join('(%s, %s, %s)' % c for c in coords)) + | 'format' >> beam.Map(format_result) | WriteToText(known_args.coordinate_output)) # Optionally render the image and save it to a file. diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test.py index 91c75aa57927..130978a9d850 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test.py @@ -23,7 +23,6 @@ import tempfile import unittest - from apache_beam.examples.complete.juliaset.juliaset import juliaset from apache_beam.testing.util import open_shards @@ -36,7 +35,7 @@ def setUp(self): self.test_files['output_image_file_name'] = self.generate_temp_file() def tearDown(self): - for test_file in self.test_files.values(): + for test_file in list(self.test_files.values()): if os.path.exists(test_file): os.remove(test_file) diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py index 0db5431e31c3..1d521bec2e4b 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py @@ -49,10 +49,8 @@ import logging - from apache_beam.examples.complete.juliaset.juliaset import juliaset - if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) juliaset.run() diff --git a/sdks/python/apache_beam/examples/complete/juliaset/setup.py b/sdks/python/apache_beam/examples/complete/juliaset/setup.py index 589e47c83059..cbf5f3d1e58d 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/setup.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/setup.py @@ -24,9 +24,10 @@ This behavior is triggered by specifying the --setup_file command line option when running the workflow for remote execution. """ +from __future__ import print_function -from distutils.command.build import build as _build import subprocess +from distutils.command.build import build as _build import setuptools @@ -76,14 +77,14 @@ def finalize_options(self): pass def RunCustomCommand(self, command_list): - print 'Running command: %s' % command_list + print('Running command: %s' % command_list) p = subprocess.Popen( command_list, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.STDOUT) # Can use communicate(input='y\n'.encode()) if the command run requires # some confirmation. stdout_data, _ = p.communicate() - print 'Command output: %s' % stdout_data + print('Command output: %s' % stdout_data) if p.returncode != 0: raise RuntimeError( 'Command %s failed: exit code: %s' % (command_list, p.returncode)) diff --git a/sdks/python/apache_beam/examples/complete/tfidf.py b/sdks/python/apache_beam/examples/complete/tfidf.py index a88ff827766c..1c97e43680e3 100644 --- a/sdks/python/apache_beam/examples/complete/tfidf.py +++ b/sdks/python/apache_beam/examples/complete/tfidf.py @@ -21,19 +21,19 @@ http://en.wikipedia.org/wiki/Tf-idf """ -from __future__ import absolute_import +from __future__ import absolute_import, division import argparse import glob import math import re +from past.utils import old_div + import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions from apache_beam.pvalue import AsSingleton -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions def read_documents(pipeline, uris): @@ -68,7 +68,8 @@ def expand(self, uri_to_content): # Create a collection of pairs mapping a URI to each of the words # in the document associated with that that URI. - def split_into_words((uri, line)): + def split_into_words(xxx_todo_changeme): + (uri, line) = xxx_todo_changeme return [(uri, w.lower()) for w in re.findall(r'[A-Za-z\']+', line)] uri_to_words = ( @@ -99,10 +100,12 @@ def split_into_words((uri, line)): # Adjust the above collection to a mapping from (URI, word) pairs to counts # into an isomorphic mapping from URI to (word, count) pairs, to prepare # for a join by the URI key. + def shift_keys(uri_word_count): + return (uri_word_count[0][0], (uri_word_count[0][1], uri_word_count[1])) + uri_to_word_and_count = ( uri_and_word_to_count - | 'ShiftKeys' >> beam.Map( - lambda ((uri, word), count): (uri, (word, count)))) + | 'ShiftKeys' >> beam.Map(shift_keys)) # Perform a CoGroupByKey (a sort of pre-join) on the prepared # uri_to_word_total and uri_to_word_and_count tagged by 'word totals' and @@ -125,12 +128,13 @@ def split_into_words((uri, line)): # that word occurs in the document divided by the total number of words in # the document. - def compute_term_frequency((uri, count_and_total)): + def compute_term_frequency(xxx_todo_changeme1): + (uri, count_and_total) = xxx_todo_changeme1 word_and_count = count_and_total['word counts'] # We have an iterable for one element that we want extracted. [word_total] = count_and_total['word totals'] for word, count in word_and_count: - yield word, (uri, float(count) / word_total) + yield word, (uri, old_div(float(count), word_total)) word_to_uri_and_tf = ( uri_to_word_and_count_and_total @@ -150,7 +154,7 @@ def compute_term_frequency((uri, count_and_total)): word_to_df = ( word_to_doc_count | 'ComputeDocFrequencies' >> beam.Map( - lambda (word, count), total: (word, float(count) / total), + lambda (word, count), total: (word, old_div(float(count), total)), AsSingleton(total_documents))) # Join the term frequency and document frequency collections, @@ -165,10 +169,11 @@ def compute_term_frequency((uri, count_and_total)): # basic version that is the term frequency divided by the log of the # document frequency. - def compute_tf_idf((word, tf_and_df)): + def compute_tf_idf(xxx_todo_changeme2): + (word, tf_and_df) = xxx_todo_changeme2 [docf] = tf_and_df['df'] for uri, tf in tf_and_df['tf']: - yield word, (uri, tf * math.log(1 / docf)) + yield word, (uri, tf * math.log(old_div(1, docf))) word_to_uri_and_tfidf = ( word_to_uri_and_tf_and_df diff --git a/sdks/python/apache_beam/examples/complete/tfidf_test.py b/sdks/python/apache_beam/examples/complete/tfidf_test.py index b6f88255887c..667951ff0d23 100644 --- a/sdks/python/apache_beam/examples/complete/tfidf_test.py +++ b/sdks/python/apache_beam/examples/complete/tfidf_test.py @@ -26,10 +26,7 @@ import apache_beam as beam from apache_beam.examples.complete import tfidf from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to -from apache_beam.testing.util import open_shards - +from apache_beam.testing.util import assert_that, equal_to, open_shards EXPECTED_RESULTS = set([ ('ghi', '1.txt', 0.3662040962227032), @@ -56,10 +53,14 @@ def test_tfidf_transform(self): [('1.txt', 'abc def ghi'), ('2.txt', 'abc def'), ('3.txt', 'abc')]) + + def re_key(word_uri_tfidf): + return (word_uri_tfidf[0], word_uri_tfidf[1][0], word_uri_tfidf[1][1]) + result = ( uri_to_line | tfidf.TfIdf() - | beam.Map(lambda (word, (uri, tfidf)): (word, uri, tfidf))) + | beam.Map(re_key)) assert_that(result, equal_to(EXPECTED_RESULTS)) # Run the pipeline. Note that the assert_that above adds to the pipeline # a check that the result PCollection contains expected values. diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py index 9a9ad7865c2c..0813bd01b521 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py @@ -47,14 +47,10 @@ import apache_beam as beam from apache_beam import combiners -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.window import Sessions -from apache_beam.transforms.window import TimestampedValue -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions - +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions +from apache_beam.transforms.window import (FixedWindows, Sessions, + TimestampedValue) ONE_HOUR_IN_SECONDS = 3600 THIRTY_DAYS_IN_SECONDS = 30 * 24 * ONE_HOUR_IN_SECONDS diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py index ced8a44af13c..bdf65be9f9c1 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py @@ -20,12 +20,10 @@ import json import unittest - import apache_beam as beam from apache_beam.examples.complete import top_wikipedia_sessions from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class ComputeTopSessionsTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py index 9911a6716bd8..380b3cd29810 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py @@ -29,15 +29,13 @@ import argparse import logging +from builtins import range from random import randrange import apache_beam as beam - from apache_beam.io import WriteToText -from apache_beam.pvalue import AsList -from apache_beam.pvalue import AsSingleton -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions +from apache_beam.pvalue import AsList, AsSingleton def create_groups(group_ids, corpus, word, ignore_corpus, ignore_word): @@ -47,7 +45,7 @@ def attach_corpus_fn(group, corpus, ignore): selected = None len_corpus = len(corpus) while not selected: - c = corpus[randrange(0, len_corpus - 1)].values()[0] + c = list(corpus[randrange(0, len_corpus - 1)].values())[0] if c != ignore: selected = c @@ -57,7 +55,7 @@ def attach_word_fn(group, words, ignore): selected = None len_words = len(words) while not selected: - c = words[randrange(0, len_words - 1)].values()[0] + c = list(words[randrange(0, len_words - 1)].values())[0] if c != ignore: selected = c @@ -91,7 +89,7 @@ def run(argv=None): with beam.Pipeline(options=pipeline_options) as p: group_ids = [] - for i in xrange(0, int(known_args.num_groups)): + for i in range(0, int(known_args.num_groups)): group_ids.append('id' + str(i)) query_corpus = 'select UNIQUE(corpus) from publicdata:samples.shakespeare' diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py index 964b35b3f08f..2be7abaf21ff 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.cookbook import bigquery_side_input from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class BigQuerySideInputTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py index 1ca49c57d592..7b40353c0a8d 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py @@ -58,7 +58,7 @@ def count_tornadoes(input_data): lambda row: [(int(row['month']), 1)] if row['tornado'] else []) | 'monthly count' >> beam.CombinePerKey(sum) | 'format' >> beam.Map( - lambda (k, v): {'month': k, 'tornado_count': v})) + lambda k_v: {'month': k_v[0], 'tornado_count': k_v[1]})) def run(argv=None): diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py index 05ee3c53f48f..961284908046 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py @@ -25,8 +25,8 @@ from nose.plugins.attrib import attr from apache_beam.examples.cookbook import bigquery_tornadoes -from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryMatcher from apache_beam.io.gcp.tests import utils +from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryMatcher from apache_beam.testing.pipeline_verifiers import PipelineStateMatcher from apache_beam.testing.test_pipeline import TestPipeline diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py index 45dcabaf853c..2d12db5050f5 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.cookbook import bigquery_tornadoes from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class BigQueryTornadoesTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/coders.py b/sdks/python/apache_beam/examples/cookbook/coders.py index f97b0f2d9c9a..0ed023715338 100644 --- a/sdks/python/apache_beam/examples/cookbook/coders.py +++ b/sdks/python/apache_beam/examples/cookbook/coders.py @@ -33,12 +33,11 @@ import argparse import json import logging +from builtins import object import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions class JsonCoder(object): diff --git a/sdks/python/apache_beam/examples/cookbook/coders_test.py b/sdks/python/apache_beam/examples/cookbook/coders_test.py index 988d3c9d25e1..9bad505cea32 100644 --- a/sdks/python/apache_beam/examples/cookbook/coders_test.py +++ b/sdks/python/apache_beam/examples/cookbook/coders_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.cookbook import coders from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class CodersTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/combiners_test.py b/sdks/python/apache_beam/examples/cookbook/combiners_test.py index ee1fb7765d47..26c9f60b256b 100644 --- a/sdks/python/apache_beam/examples/cookbook/combiners_test.py +++ b/sdks/python/apache_beam/examples/cookbook/combiners_test.py @@ -28,8 +28,7 @@ import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class CombinersTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py b/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py index aee69d23ff65..282773293f76 100644 --- a/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py +++ b/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py @@ -26,14 +26,13 @@ import logging import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText +from apache_beam.io import ReadFromText, WriteToText from apache_beam.options.pipeline_options import PipelineOptions - # pylint doesn't understand our pipeline syntax: # pylint:disable=expression-not-assigned + class Count1(beam.PTransform): """Count as a subclass of PTransform, with an apply method.""" diff --git a/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py b/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py index 7aaccb4ac1c0..88d83fe2a6ea 100644 --- a/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py +++ b/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.cookbook import custom_ptransform from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class CustomCountTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py index 7161cff1c255..63ebd0f37210 100644 --- a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py +++ b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py @@ -65,20 +65,21 @@ import logging import re import uuid +from builtins import object -from google.cloud.proto.datastore.v1 import entity_pb2 -from google.cloud.proto.datastore.v1 import query_pb2 -from googledatastore import helper as datastore_helper, PropertyFilter +from google.cloud.proto.datastore.v1 import entity_pb2, query_pb2 +from googledatastore import helper as datastore_helper +from googledatastore import PropertyFilter import apache_beam as beam from apache_beam.io import ReadFromText -from apache_beam.io.gcp.datastore.v1.datastoreio import ReadFromDatastore -from apache_beam.io.gcp.datastore.v1.datastoreio import WriteToDatastore +from apache_beam.io.gcp.datastore.v1.datastoreio import (ReadFromDatastore, + WriteToDatastore) from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import (GoogleCloudOptions, + PipelineOptions, + SetupOptions) class WordExtractingDoFn(beam.DoFn): @@ -129,7 +130,7 @@ def make_entity(self, content): datastore_helper.add_key_path(entity.key, self._kind, self._ancestor, self._kind, str(uuid.uuid4())) - datastore_helper.add_properties(entity, {"content": unicode(content)}) + datastore_helper.add_properties(entity, {"content": str(content)}) return entity @@ -178,15 +179,21 @@ def read_from_datastore(project, user_options, pipeline_options): project, query, user_options.namespace) # Count the occurrences of each word. + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + counts = (lines | 'split' >> (beam.ParDo(WordExtractingDoFn()) - .with_output_types(unicode)) + .with_output_types(str)) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() - | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones)))) + | 'count' >> beam.Map(sum_word_counts)) # Format the counts into a PCollection of strings. - output = counts | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + def format_result(word_c): + return '%s: %s' % (word_c[0], word_c[1]) + + output = counts | 'format' >> beam.Map(format_result) # Write the output using a "Write" transform that has side effects. # pylint: disable=expression-not-assigned diff --git a/sdks/python/apache_beam/examples/cookbook/filters_test.py b/sdks/python/apache_beam/examples/cookbook/filters_test.py index fd49f938742c..ff70caf7cba7 100644 --- a/sdks/python/apache_beam/examples/cookbook/filters_test.py +++ b/sdks/python/apache_beam/examples/cookbook/filters_test.py @@ -23,8 +23,7 @@ import apache_beam as beam from apache_beam.examples.cookbook import filters from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class FiltersTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/cookbook/group_with_coder.py b/sdks/python/apache_beam/examples/cookbook/group_with_coder.py index 9c0d04b816ac..16d642e01d8d 100644 --- a/sdks/python/apache_beam/examples/cookbook/group_with_coder.py +++ b/sdks/python/apache_beam/examples/cookbook/group_with_coder.py @@ -30,15 +30,14 @@ import argparse import logging import sys +from builtins import object import apache_beam as beam from apache_beam import coders -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions from apache_beam.typehints import typehints from apache_beam.typehints.decorators import with_output_types -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions class Player(object): @@ -114,7 +113,7 @@ def run(args=None): # is registered for the Player class above, a PlayerCoder will be used to # encode Player objects as keys for this combine operation. | beam.CombinePerKey(sum) - | beam.Map(lambda (k, v): '%s,%d' % (k.name, v)) + | beam.Map(lambda k_v: '%s,%d' % (k_v[0].name, k_v[1])) | WriteToText(known_args.output)) diff --git a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py index fb630ba465f7..ed38b5d8a4ed 100644 --- a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py +++ b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py @@ -24,7 +24,6 @@ from apache_beam.examples.cookbook import group_with_coder from apache_beam.testing.util import open_shards - # Patch group_with_coder.PlayerCoder.decode(). To test that the PlayerCoder was # used, we do not strip the prepended 'x:' string when decoding a Player object. group_with_coder.PlayerCoder.decode = lambda self, s: group_with_coder.Player(s) diff --git a/sdks/python/apache_beam/examples/cookbook/mergecontacts.py b/sdks/python/apache_beam/examples/cookbook/mergecontacts.py index 9acdd9073478..1a9af40cb4ee 100644 --- a/sdks/python/apache_beam/examples/cookbook/mergecontacts.py +++ b/sdks/python/apache_beam/examples/cookbook/mergecontacts.py @@ -34,14 +34,12 @@ import argparse import logging import re +from builtins import next import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions +from apache_beam.testing.util import assert_that, equal_to def run(argv=None, assert_results=None): @@ -96,19 +94,28 @@ def read_kv_textfile(label, textfile): # Prepare tab-delimited output; something like this: # "name""email_1,email_2""phone""first_snailmail_only" tsv_lines = grouped | beam.Map( - lambda (name, (email, phone, snailmail)): '\t'.join( - ['"%s"' % name, - '"%s"' % ','.join(email), - '"%s"' % ','.join(phone), - '"%s"' % next(iter(snailmail), '')])) + lambda name_email_phone_snailmail: '\t'.join( + ['"%s"' % name_email_phone_snailmail[0], + '"%s"' % ','.join(name_email_phone_snailmail[1][0]), + '"%s"' % ','.join(name_email_phone_snailmail[1][1]), + '"%s"' % next(iter(name_email_phone_snailmail[1][2]), '')])) # Compute some stats about our database of people. + def without_email(name_email_phone_snailmail1): + return not next(iter(name_email_phone_snailmail1[1][0]), None) + + def without_phones(name_email_phone_snailmail2): + return not next(iter(name_email_phone_snailmail2[1][1]), None) + + def without_address(name_e_p_snailmail): + return not next(iter(name_e_p_snailmail[1][2]), None) + luddites = grouped | beam.Filter( # People without email. - lambda (name, (email, phone, snailmail)): not next(iter(email), None)) + without_email) writers = grouped | beam.Filter( # People without phones. - lambda (name, (email, phone, snailmail)): not next(iter(phone), None)) + without_phones) nomads = grouped | beam.Filter( # People without addresses. - lambda (name, (e, p, snailmail)): not next(iter(snailmail), None)) + without_address) num_luddites = luddites | 'Luddites' >> beam.combiners.Count.Globally() num_writers = writers | 'Writers' >> beam.combiners.Count.Globally() diff --git a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py index 2316c6611c06..5bc899144ed2 100644 --- a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py +++ b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py @@ -57,10 +57,8 @@ import apache_beam as beam from apache_beam import pvalue -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions class SplitLinesToWordsFn(beam.DoFn): @@ -119,11 +117,17 @@ class CountWords(beam.PTransform): """ def expand(self, pcoll): + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + + def format_result(word_c): + return '%s: %s' % (word_c[0], word_c[1]) + return (pcoll | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() - | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones))) - | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c))) + | 'count' >> beam.Map(sum_word_counts) + | 'format' >> beam.Map(format_result)) def run(argv=None): @@ -163,7 +167,7 @@ def run(argv=None): (character_count | 'pair_with_key' >> beam.Map(lambda x: ('chars_temp_key', x)) | beam.GroupByKey() - | 'count chars' >> beam.Map(lambda (_, counts): sum(counts)) + | 'count chars' >> beam.Map(lambda __counts: sum(__counts[1])) | 'write chars' >> WriteToText(known_args.output + '-chars')) # pylint: disable=expression-not-assigned diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 27b81208fcd8..9b6aefdf642f 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -30,11 +30,16 @@ string. The tags can contain only letters, digits and _. """ +from builtins import object, range + import apache_beam as beam +from apache_beam.io import iobase +from apache_beam.io.range_trackers import OffsetRangeTracker from apache_beam.metrics import Metrics +from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to +from apache_beam.transforms.core import PTransform # Quiet some pylint warnings that happen because of the somewhat special # format for the code snippets. @@ -432,7 +437,7 @@ def examples_wordcount_minimal(renames): # [END examples_wordcount_minimal_count] # [START examples_wordcount_minimal_map] - | beam.Map(lambda (word, count): '%s: %s' % (word, count)) + | beam.Map(lambda word_count: '%s: %s' % (word_count[0], word_count[1])) # [END examples_wordcount_minimal_map] # [START examples_wordcount_minimal_write] @@ -531,14 +536,19 @@ def _add_argparse_args(cls, parser): lines = p | 'Read' >> ReadFromText(wordcount_options.input) # [END example_wordcount_templated] + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + + def format_result(word_c2): + return '%s: %s' % (word_c2[0], word_c2[1]) ( lines | 'ExtractWords' >> beam.FlatMap( lambda x: re.findall(r'[A-Za-z\']+', x)) | 'PairWithOnes' >> beam.Map(lambda x: (x, 1)) | 'Group' >> beam.GroupByKey() - | 'Sum' >> beam.Map(lambda (word, ones): (word, sum(ones))) - | 'Format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'Sum' >> beam.Map(sum_word_counts) + | 'Format' >> beam.Map(format_result) | 'Write' >> WriteToText(wordcount_options.output) ) @@ -607,8 +617,11 @@ def process(self, element): [('Flourish', 3), ('stomach', 1)])) # [END example_wordcount_debugging_assert] + def format_result(word_c1): + return '%s: %s' % (word_c1[0], word_c1[1]) + output = (filtered_words - | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'format' >> beam.Map(format_result) | 'Write' >> beam.io.WriteToText('gs://my-bucket/counts.txt')) p.visit(SnippetUtils.RenameFiles(renames)) @@ -655,13 +668,6 @@ def process(self, an_int): result.wait_until_finish() -import apache_beam as beam -from apache_beam.io import iobase -from apache_beam.io.range_trackers import OffsetRangeTracker -from apache_beam.transforms.core import PTransform -from apache_beam.options.pipeline_options import PipelineOptions - - # Defining a new source. # [START model_custom_source_new_source] class CountingSource(iobase.BoundedSource): @@ -976,7 +982,7 @@ def model_datastoreio(): def to_entity(content): entity = entity_pb2.Entity() googledatastore.helper.add_key_path(entity.key, kind, str(uuid.uuid4())) - googledatastore.helper.add_properties(entity, {'content': unicode(content)}) + googledatastore.helper.add_properties(entity, {'content': str(content)}) return entity entities = musicians | 'To Entity' >> beam.Map(to_entity) @@ -1049,7 +1055,7 @@ def expand(self, pcoll): return (pcoll | beam.FlatMap(lambda x: re.findall(r'\w+', x)) | beam.combiners.Count.PerElement() - | beam.Map(lambda (word, c): '%s: %s' % (word, c))) + | beam.Map(lambda word_c: '%s: %s' % (word_c[0], word_c[1]))) # [END composite_ptransform_apply_method] # [END composite_transform_example] @@ -1111,7 +1117,7 @@ def partition_fn(student, num_partitions): fortieth_percentile = by_decile[4] # [END model_multiple_pcollections_partition_40th] - ([by_decile[d] for d in xrange(10) if d != 4] + [fortieth_percentile] + ([by_decile[d] for d in range(10) if d != 4] + [fortieth_percentile] | beam.Flatten() | beam.io.WriteToText(output_path)) @@ -1135,8 +1141,12 @@ def model_group_by_key(contents, output_path): # [START model_group_by_key_transform] grouped_words = words_and_counts | beam.GroupByKey() # [END model_group_by_key_transform] + + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + (grouped_words - | 'count words' >> beam.Map(lambda (word, counts): (word, sum(counts))) + | 'count words' >> beam.Map(sum_word_counts) | beam.io.WriteToText(output_path)) @@ -1163,7 +1173,8 @@ def model_co_group_by_key_tuple(email_list, phone_list, output_path): # ('joe', {'emails': ['joe@example.com', 'joe@gmail.com'], 'phones': ...}) result = {'emails': emails, 'phones': phones} | beam.CoGroupByKey() - def join_info((name, info)): + def join_info(xxx_todo_changeme): + (name, info) = xxx_todo_changeme return '; '.join(['%s' % name, '%s' % ','.join(info['emails']), '%s' % ','.join(info['phones'])]) @@ -1214,7 +1225,7 @@ def join_info(name, emails, phone_numbers): class Keys(beam.PTransform): def expand(self, pcoll): - return pcoll | 'Keys' >> beam.Map(lambda (k, v): k) + return pcoll | 'Keys' >> beam.Map(lambda k_v: k_v[0]) # [END model_library_transforms_keys] # pylint: enable=invalid-name diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index 9183d0dfea19..15d762a5687b 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -16,30 +16,37 @@ # """Tests for all code snippets used in public docs.""" +from __future__ import division import glob import gzip import logging import os +import sys import tempfile import unittest import uuid +from builtins import map, object, range, zip + +from past.utils import old_div import apache_beam as beam -from apache_beam import coders -from apache_beam import pvalue -from apache_beam import typehints +from apache_beam import coders, pvalue, typehints from apache_beam.coders.coders import ToStringCoder -from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.examples.snippets import snippets from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to -from apache_beam.utils.windowed_value import WindowedValue - +from apache_beam.options.pipeline_options import PipelineOptions # pylint: disable=expression-not-assigned from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that, equal_to +from apache_beam.utils.windowed_value import WindowedValue + +if sys.version_info[0] >= 3: + newint = int +else: + from builtins import int as newint + # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -304,7 +311,7 @@ def expand(self, pcoll): # pylint: disable=expression-not-assigned with self.assertRaises(typehints.TypeCheckError): words_with_lens | beam.Map(lambda x: x).with_input_types( - beam.typehints.Tuple[int, int]) + beam.typehints.Tuple[newint, newint]) def test_runtime_checks_off(self): # pylint: disable=expression-not-assigned @@ -356,11 +363,11 @@ def parse_player_and_score(csv): lines | beam.Map(parse_player_and_score) | beam.CombinePerKey(sum).with_input_types( - beam.typehints.Tuple[Player, int])) + beam.typehints.Tuple[Player, newint])) # [END type_hints_deterministic_key] assert_that( - totals | beam.Map(lambda (k, v): (k.name, v)), + totals | beam.Map(lambda k_v: (k_v[0].name, k_v[1])), equal_to([('banana', 3), ('kiwi', 4), ('zucchini', 3)])) @@ -454,7 +461,7 @@ def tearDown(self): beam.io.ReadFromText = self.old_read_from_text beam.io.WriteToText = self.old_write_to_text # Cleanup all the temporary files created in the test - map(os.remove, self.temp_files) + list(map(os.remove, self.temp_files)) def create_temp_file(self, contents=''): with tempfile.NamedTemporaryFile(delete=False) as f: @@ -846,15 +853,17 @@ class AverageFn(beam.CombineFn): def create_accumulator(self): return (0.0, 0) - def add_input(self, (sum, count), input): + def add_input(self, xxx_todo_changeme, input): + (sum, count) = xxx_todo_changeme return sum + input, count + 1 def merge_accumulators(self, accumulators): - sums, counts = zip(*accumulators) + sums, counts = list(zip(*accumulators)) return sum(sums), sum(counts) - def extract_output(self, (sum, count)): - return sum / count if count else float('NaN') + def extract_output(self, xxx_todo_changeme1): + (sum, count) = xxx_todo_changeme1 + return old_div(sum, count) if count else float('NaN') # [END combine_custom_average_define] # [START combine_custom_average_execute] average = pc | beam.CombineGlobally(AverageFn()) diff --git a/sdks/python/apache_beam/examples/streaming_wordcount.py b/sdks/python/apache_beam/examples/streaming_wordcount.py index 7696d7789323..50151f38cc9d 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcount.py +++ b/sdks/python/apache_beam/examples/streaming_wordcount.py @@ -26,11 +26,10 @@ import argparse import logging - import apache_beam as beam -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import StandardOptions import apache_beam.transforms.window as window +from apache_beam.options.pipeline_options import (PipelineOptions, + StandardOptions) def split_fn(lines): @@ -59,14 +58,16 @@ def run(argv=None): lines = p | beam.io.ReadStringsFromPubSub(known_args.input_topic) # Capitalize the characters in each line. + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) transformed = (lines # Use a pre-defined function that imports the re package. | 'Split' >> ( - beam.FlatMap(split_fn).with_output_types(unicode)) + beam.FlatMap(split_fn).with_output_types(str)) | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) | beam.WindowInto(window.FixedWindows(15, 0)) | 'Group' >> beam.GroupByKey() - | 'Count' >> beam.Map(lambda (word, ones): (word, sum(ones))) + | 'Count' >> beam.Map(sum_word_counts) | 'Format' >> beam.Map(lambda tup: '%s: %d' % tup)) # Write to PubSub. diff --git a/sdks/python/apache_beam/examples/windowed_wordcount.py b/sdks/python/apache_beam/examples/windowed_wordcount.py index bd57847c67f9..c10eb2dbde40 100644 --- a/sdks/python/apache_beam/examples/windowed_wordcount.py +++ b/sdks/python/apache_beam/examples/windowed_wordcount.py @@ -26,7 +26,6 @@ import argparse import logging - import apache_beam as beam import apache_beam.transforms.window as window @@ -70,13 +69,16 @@ def run(argv=None): lines = p | beam.io.ReadStringsFromPubSub(known_args.input_topic) # Capitalize the characters in each line. + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + transformed = (lines | 'Split' >> (beam.FlatMap(find_words) - .with_output_types(unicode)) + .with_output_types(str)) | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) | beam.WindowInto(window.FixedWindows(2*60, 0)) | 'Group' >> beam.GroupByKey() - | 'Count' >> beam.Map(lambda (word, ones): (word, sum(ones))) + | 'Count' >> beam.Map(sum_word_counts) | 'Format' >> beam.ParDo(FormatDoFn())) # Write to BigQuery. diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py index 34dedb2b819a..54a779745ffb 100644 --- a/sdks/python/apache_beam/examples/wordcount.py +++ b/sdks/python/apache_beam/examples/wordcount.py @@ -24,12 +24,10 @@ import re import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText +from apache_beam.io import ReadFromText, WriteToText from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions class WordExtractingDoFn(beam.DoFn): @@ -88,15 +86,21 @@ def run(argv=None): lines = p | 'read' >> ReadFromText(known_args.input) # Count the occurrences of each word. + def sum_word_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + counts = (lines | 'split' >> (beam.ParDo(WordExtractingDoFn()) - .with_output_types(unicode)) + .with_output_types(str)) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() - | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones)))) + | 'count' >> beam.Map(sum_word_counts)) # Format the counts into a PCollection of strings. - output = counts | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + def format_result(word_c): + return '%s: %s' % (word_c[0], word_c[1]) + + output = counts | 'format' >> beam.Map(format_result) # Write the output using a "Write" transform that has side effects. # pylint: disable=expression-not-assigned diff --git a/sdks/python/apache_beam/examples/wordcount_debugging.py b/sdks/python/apache_beam/examples/wordcount_debugging.py index c0ffd356364c..9ec9d1bc9562 100644 --- a/sdks/python/apache_beam/examples/wordcount_debugging.py +++ b/sdks/python/apache_beam/examples/wordcount_debugging.py @@ -46,13 +46,10 @@ import re import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText +from apache_beam.io import ReadFromText, WriteToText from apache_beam.metrics import Metrics -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions +from apache_beam.testing.util import assert_that, equal_to class FilterTextFn(beam.DoFn): @@ -93,12 +90,15 @@ class CountWords(beam.PTransform): PCollection of (word, count) tuples. """ def expand(self, pcoll): + def sum_counts(word_ones): + return (word_ones[0], sum(word_ones[1])) + return (pcoll | 'split' >> (beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - .with_output_types(unicode)) + .with_output_types(str)) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() - | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones)))) + | 'count' >> beam.Map(sum_counts)) def run(argv=None): @@ -138,11 +138,13 @@ def run(argv=None): assert_that( filtered_words, equal_to([('Flourish', 3), ('stomach', 1)])) + def format_result(word_c): + return '%s: %s' % (word_c[0], word_c[1]) # Format the counts into a PCollection of strings and write the output using # a "Write" transform that has side effects. # pylint: disable=unused-variable output = (filtered_words - | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'format' >> beam.Map(format_result) | 'write' >> WriteToText(known_args.output)) diff --git a/sdks/python/apache_beam/examples/wordcount_it_test.py b/sdks/python/apache_beam/examples/wordcount_it_test.py index 8d2e73e28f6b..dc8e8482c80c 100644 --- a/sdks/python/apache_beam/examples/wordcount_it_test.py +++ b/sdks/python/apache_beam/examples/wordcount_it_test.py @@ -25,10 +25,10 @@ from nose.plugins.attrib import attr from apache_beam.examples import wordcount -from apache_beam.testing.test_utils import delete_files -from apache_beam.testing.pipeline_verifiers import FileChecksumMatcher -from apache_beam.testing.pipeline_verifiers import PipelineStateMatcher +from apache_beam.testing.pipeline_verifiers import (FileChecksumMatcher, + PipelineStateMatcher) from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_utils import delete_files class WordCountIT(unittest.TestCase): diff --git a/sdks/python/apache_beam/examples/wordcount_minimal.py b/sdks/python/apache_beam/examples/wordcount_minimal.py index 76b0a221df7a..f0b30109c5d9 100644 --- a/sdks/python/apache_beam/examples/wordcount_minimal.py +++ b/sdks/python/apache_beam/examples/wordcount_minimal.py @@ -51,10 +51,8 @@ import re import apache_beam as beam -from apache_beam.io import ReadFromText -from apache_beam.io import WriteToText -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.io import ReadFromText, WriteToText +from apache_beam.options.pipeline_options import PipelineOptions, SetupOptions def run(argv=None): @@ -101,12 +99,14 @@ def run(argv=None): counts = ( lines | 'Split' >> (beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - .with_output_types(unicode)) + .with_output_types(str)) | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) | 'GroupAndSum' >> beam.CombinePerKey(sum)) # Format the counts into a PCollection of strings. - output = counts | 'Format' >> beam.Map(lambda (w, c): '%s: %s' % (w, c)) + def format_result(w_c): + return '%s: %s' % (w_c[0], w_c[1]) + output = counts | 'Format' >> beam.Map(format_result) # Write the output using a "Write" transform that has side effects. # pylint: disable=expression-not-assigned diff --git a/sdks/python/apache_beam/examples/wordcount_minimal_test.py b/sdks/python/apache_beam/examples/wordcount_minimal_test.py index 5ee7b7803980..b361ce39289f 100644 --- a/sdks/python/apache_beam/examples/wordcount_minimal_test.py +++ b/sdks/python/apache_beam/examples/wordcount_minimal_test.py @@ -52,7 +52,7 @@ def test_basics(self): match = re.search(r'([a-z]+): ([0-9]+)', line) if match is not None: results.append((match.group(1), int(match.group(2)))) - self.assertEqual(sorted(results), sorted(expected_words.iteritems())) + self.assertEqual(sorted(results), sorted(expected_words.items())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/wordcount_test.py b/sdks/python/apache_beam/examples/wordcount_test.py index 9834ba53111a..8ffa7733b780 100644 --- a/sdks/python/apache_beam/examples/wordcount_test.py +++ b/sdks/python/apache_beam/examples/wordcount_test.py @@ -51,7 +51,7 @@ def test_basics(self): match = re.search(r'([a-z]+): ([0-9]+)', line) if match is not None: results.append((match.group(1), int(match.group(2)))) - self.assertEqual(sorted(results), sorted(expected_words.iteritems())) + self.assertEqual(sorted(results), sorted(expected_words.items())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py index 9f32092f4166..ffe7d52c65e1 100644 --- a/sdks/python/apache_beam/internal/gcp/auth.py +++ b/sdks/python/apache_beam/internal/gcp/auth.py @@ -21,13 +21,17 @@ import json import logging import os -import urllib2 +import urllib.error +import urllib.parse +import urllib.request -from oauth2client.client import GoogleCredentials -from oauth2client.client import OAuth2Credentials +from future import standard_library +from oauth2client.client import GoogleCredentials, OAuth2Credentials from apache_beam.utils import retry +standard_library.install_aliases() + # When we are running in GCE, we can authenticate with VM credentials. is_running_in_gce = False @@ -90,8 +94,9 @@ def _refresh(self, http_request): 'GCE_METADATA_ROOT', 'metadata.google.internal') token_url = ('http://{}/computeMetadata/v1/instance/service-accounts/' 'default/token').format(metadata_root) - req = urllib2.Request(token_url, headers={'Metadata-Flavor': 'Google'}) - token_data = json.loads(urllib2.urlopen(req).read()) + req = urllib.request.Request( + token_url, headers={'Metadata-Flavor': 'Google'}) + token_data = json.loads(urllib.request.urlopen(req).read()) self.access_token = token_data['access_token'] self.token_expiry = (refresh_time + datetime.timedelta(seconds=token_data['expires_in'])) diff --git a/sdks/python/apache_beam/internal/gcp/json_value.py b/sdks/python/apache_beam/internal/gcp/json_value.py index 167b173b1f84..ad9b760b0b85 100644 --- a/sdks/python/apache_beam/internal/gcp/json_value.py +++ b/sdks/python/apache_beam/internal/gcp/json_value.py @@ -19,14 +19,16 @@ # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position +from past.builtins import basestring + +from apache_beam.options.value_provider import ValueProvider + try: from apitools.base.py import extra_types except ImportError: extra_types = None # pylint: enable=wrong-import-order, wrong-import-position -from apache_beam.options.value_provider import ValueProvider - _MAXINT64 = (1 << 63) - 1 _MININT64 = - (1 << 63) @@ -93,7 +95,7 @@ def to_json_value(obj, with_type=False): entries=[to_json_value(o, with_type=with_type) for o in obj])) elif isinstance(obj, dict): json_object = extra_types.JsonObject() - for k, v in obj.iteritems(): + for k, v in obj.items(): json_object.properties.append( extra_types.JsonObject.Property( key=k, value=to_json_value(v, with_type=with_type))) @@ -106,7 +108,7 @@ def to_json_value(obj, with_type=False): return extra_types.JsonValue(boolean_value=obj) elif isinstance(obj, int): return extra_types.JsonValue(integer_value=obj) - elif isinstance(obj, long): + elif isinstance(obj, int): if _MININT64 <= obj <= _MAXINT64: return extra_types.JsonValue(integer_value=obj) else: diff --git a/sdks/python/apache_beam/internal/gcp/json_value_test.py b/sdks/python/apache_beam/internal/gcp/json_value_test.py index b1fd63f048e9..723431e75781 100644 --- a/sdks/python/apache_beam/internal/gcp/json_value_test.py +++ b/sdks/python/apache_beam/internal/gcp/json_value_test.py @@ -19,11 +19,9 @@ import unittest -from apache_beam.internal.gcp.json_value import from_json_value -from apache_beam.internal.gcp.json_value import to_json_value -from apache_beam.options.value_provider import StaticValueProvider -from apache_beam.options.value_provider import RuntimeValueProvider - +from apache_beam.internal.gcp.json_value import from_json_value, to_json_value +from apache_beam.options.value_provider import (RuntimeValueProvider, + StaticValueProvider) # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -90,14 +88,14 @@ def test_none_from(self): def test_large_integer(self): num = 1 << 35 self.assertEquals(num, from_json_value(to_json_value(num))) - self.assertEquals(long(num), from_json_value(to_json_value(long(num)))) + self.assertEquals(int(num), from_json_value(to_json_value(int(num)))) def test_long_value(self): - self.assertEquals(long(27), from_json_value(to_json_value(long(27)))) + self.assertEquals(int(27), from_json_value(to_json_value(int(27)))) def test_too_long_value(self): with self.assertRaises(TypeError): - to_json_value(long(1 << 64)) + to_json_value(int(1 << 64)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/internal/module_test.py b/sdks/python/apache_beam/internal/module_test.py index c7bb320253ae..c74819598e15 100644 --- a/sdks/python/apache_beam/internal/module_test.py +++ b/sdks/python/apache_beam/internal/module_test.py @@ -18,6 +18,7 @@ """Module used to define functions and classes used by the coder unit tests.""" import re +from builtins import object class TopClass(object): diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py index e049a7134231..146abaed656a 100644 --- a/sdks/python/apache_beam/internal/pickler.py +++ b/sdks/python/apache_beam/internal/pickler.py @@ -49,10 +49,10 @@ def _find_containing_class(nested_class): """Finds containing class of a nestec class passed as argument.""" def _find_containing_class_inner(outer): - for k, v in outer.__dict__.items(): + for k, v in list(outer.__dict__.items()): if v is nested_class: return outer, k - elif isinstance(v, (type, types.ClassType)) and hasattr(v, '__dict__'): + elif isinstance(v, type) and hasattr(v, '__dict__'): res = _find_containing_class_inner(v) if res: return res @@ -144,7 +144,7 @@ def new_save_module_dict(pickler, obj): obj_id = id(obj) if not known_module_dicts or '__file__' in obj or '__package__' in obj: if obj_id not in known_module_dicts: - for m in sys.modules.values(): + for m in list(sys.modules.values()): try: if m and m.__name__ != '__main__': d = m.__dict__ diff --git a/sdks/python/apache_beam/internal/pickler_test.py b/sdks/python/apache_beam/internal/pickler_test.py index 05062d282da1..a88aeaa1861e 100644 --- a/sdks/python/apache_beam/internal/pickler_test.py +++ b/sdks/python/apache_beam/internal/pickler_test.py @@ -18,10 +18,10 @@ """Unit tests for the pickler module.""" import unittest +from builtins import range from apache_beam.internal import module_test -from apache_beam.internal.pickler import dumps -from apache_beam.internal.pickler import loads +from apache_beam.internal.pickler import dumps, loads class PicklerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/internal/util.py b/sdks/python/apache_beam/internal/util.py index dbbeafcc1e75..79811c0eff1e 100644 --- a/sdks/python/apache_beam/internal/util.py +++ b/sdks/python/apache_beam/internal/util.py @@ -21,9 +21,10 @@ """ import logging -from multiprocessing.pool import ThreadPool import threading import weakref +from builtins import object +from multiprocessing.pool import ThreadPool class ArgumentPlaceholder(object): @@ -79,7 +80,7 @@ def swapper(value): # by sorting the entries first. This will be important when putting back # PValues. new_kwargs = dict((k, swapper(v)) if isinstance(v, pvalue_classes) else (k, v) - for k, v in sorted(kwargs.iteritems())) + for k, v in sorted(kwargs.items())) return (new_args, new_kwargs, pvals) @@ -100,11 +101,11 @@ def insert_values_in_args(args, kwargs, values): # Use a local iterator so that we don't modify values. v_iter = iter(values) new_args = [ - v_iter.next() if isinstance(arg, ArgumentPlaceholder) else arg + next(v_iter) if isinstance(arg, ArgumentPlaceholder) else arg for arg in args] new_kwargs = dict( - (k, v_iter.next()) if isinstance(v, ArgumentPlaceholder) else (k, v) - for k, v in sorted(kwargs.iteritems())) + (k, next(v_iter)) if isinstance(v, ArgumentPlaceholder) else (k, v) + for k, v in sorted(kwargs.items())) return (new_args, new_kwargs) diff --git a/sdks/python/apache_beam/internal/util_test.py b/sdks/python/apache_beam/internal/util_test.py index 9a2e3977a708..374307ac09bb 100644 --- a/sdks/python/apache_beam/internal/util_test.py +++ b/sdks/python/apache_beam/internal/util_test.py @@ -19,9 +19,9 @@ import unittest -from apache_beam.internal.util import ArgumentPlaceholder -from apache_beam.internal.util import insert_values_in_args -from apache_beam.internal.util import remove_objects_from_args +from apache_beam.internal.util import (ArgumentPlaceholder, + insert_values_in_args, + remove_objects_from_args) class UtilTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index cb14c6542176..d0dbf9d13c24 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -41,25 +41,34 @@ that can be used to write a given ``PCollection`` of Python objects to an Avro file. """ +from __future__ import print_function -import cStringIO +import io import os import zlib +from builtins import object from functools import partial -import avro -from avro import datafile -from avro import io as avroio -from avro import schema +try: + import avro + from avro import datafile + from avro import io as avroio + from avro import schema +except ImportError as e: + print("There as an error importing avro. Verify avro-python3 is installed") + raise e + +from future import standard_library import apache_beam as beam -from apache_beam.io import filebasedsource -from apache_beam.io import filebasedsink -from apache_beam.io import iobase +from apache_beam.io import filebasedsink, filebasedsource, iobase from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.iobase import Read from apache_beam.transforms import PTransform +standard_library.install_aliases() + + __all__ = ['ReadFromAvro', 'ReadAllFromAvro', 'WriteToAvro'] @@ -311,7 +320,7 @@ def _decompress_bytes(data, codec): # We take care to avoid extra copies of data while slicing large objects # by use of a buffer. result = snappy.decompress(buffer(data)[:-4]) - avroio.BinaryDecoder(cStringIO.StringIO(data[-4:])).check_crc32(result) + avroio.BinaryDecoder(io.BytesIO(data[-4:])).check_crc32(result) return result else: raise ValueError('Unknown codec: %r', codec) @@ -321,7 +330,7 @@ def num_records(self): def records(self): decoder = avroio.BinaryDecoder( - cStringIO.StringIO(self._decompressed_block_bytes)) + io.BytesIO(self._decompressed_block_bytes)) reader = avroio.DatumReader( writers_schema=self._schema, readers_schema=self._schema) diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 969f44050854..a42fff7833f6 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -20,30 +20,24 @@ import os import tempfile import unittest +from builtins import range +import hamcrest as hc + +import avro.datafile +import avro.schema +from avro.datafile import DataFileWriter +from avro.io import DatumWriter import apache_beam as beam from apache_beam import Create -from apache_beam.io import iobase -from apache_beam.io import avroio -from apache_beam.io import filebasedsource -from apache_beam.io import source_test_utils +from apache_beam.io import avroio, filebasedsource, iobase, source_test_utils +from apache_beam.io.avroio import _AvroSink as AvroSink +from apache_beam.io.avroio import _AvroSource as AvroSource from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher -# Importing following private class for testing purposes. -from apache_beam.io.avroio import _AvroSource as AvroSource -from apache_beam.io.avroio import _AvroSink as AvroSink - -import avro.datafile -from avro.datafile import DataFileWriter -from avro.io import DatumWriter -import avro.schema -import hamcrest as hc - - # Import snappy optionally; some tests will be skipped when import fails. try: import snappy # pylint: disable=import-error diff --git a/sdks/python/apache_beam/io/concat_source.py b/sdks/python/apache_beam/io/concat_source.py index 56c4ccabcf71..7364f5445bf7 100644 --- a/sdks/python/apache_beam/io/concat_source.py +++ b/sdks/python/apache_beam/io/concat_source.py @@ -19,9 +19,13 @@ Concat Source, which reads the union of several other sources. """ +from __future__ import division import bisect import threading +from builtins import range + +from past.utils import old_div from apache_beam.io import iobase @@ -136,7 +140,7 @@ def _compute_cumulative_weights(source_bundles): # Substitute average weights for those whose weights are # unspecified (or 1.0 for everything if none are known). known = [s.weight for s in source_bundles if s.weight is not None] - avg = sum(known) / len(known) if known else 1.0 + avg = old_div(sum(known), len(known)) if known else 1.0 weights = [s.weight or avg for s in source_bundles] # Now compute running totals of the percent done upon reaching @@ -147,7 +151,7 @@ def _compute_cumulative_weights(source_bundles): running_total = [0] for w in weights: running_total.append( - max(min_diff, min(1, running_total[-1] + w / total))) + max(min_diff, min(1, running_total[-1] + old_div(w, total)))) running_total[-1] = 1 # In case of rounding error. # There are issues if, due to rouding error or greatly differing sizes, # two adjacent running total weights are equal. Normalize this things so @@ -206,7 +210,7 @@ def try_split(self, pos): ratio = self.local_to_global(source_ix, frac) self._end = source_ix, split_pos - self._cumulative_weights = [min(w / ratio, 1) + self._cumulative_weights = [min(old_div(w, ratio), 1) for w in self._cumulative_weights] return (source_ix, split_pos), ratio @@ -248,7 +252,8 @@ def global_to_local(self, frac): # this source into a value in [0.0, 1.0) representing how far we are # towards the next source. return (source_ix, - (frac - cw[source_ix]) / (cw[source_ix + 1] - cw[source_ix])) + old_div((frac - cw[source_ix]), + (cw[source_ix + 1] - cw[source_ix]))) def sub_range_tracker(self, source_ix): assert self._start[0] <= source_ix <= self._end[0] diff --git a/sdks/python/apache_beam/io/concat_source_test.py b/sdks/python/apache_beam/io/concat_source_test.py index 4a8f5193dee2..8df0ef3b7be8 100644 --- a/sdks/python/apache_beam/io/concat_source_test.py +++ b/sdks/python/apache_beam/io/concat_source_test.py @@ -16,19 +16,19 @@ # """Unit tests for the sources framework.""" +from __future__ import division import logging import unittest +from builtins import range -import apache_beam as beam +from past.utils import old_div -from apache_beam.io import iobase -from apache_beam.io import range_trackers -from apache_beam.io import source_test_utils +import apache_beam as beam +from apache_beam.io import iobase, range_trackers, source_test_utils from apache_beam.io.concat_source import ConcatSource from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class RangeSource(iobase.BoundedSource): @@ -92,10 +92,10 @@ def test_conact_source(self): RangeSource(12, 16), ]) self.assertEqual(list(source.read(source.get_range_tracker())), - range(16)) + list(range(16))) self.assertEqual(list(source.read(source.get_range_tracker((1, None), (2, 10)))), - range(4, 10)) + list(range(4, 10))) range_tracker = source.get_range_tracker(None, None) self.assertEqual(range_tracker.position_at_fraction(0), (0, 0)) self.assertEqual(range_tracker.position_at_fraction(.5), (2, 8)) @@ -119,7 +119,8 @@ def test_conact_source(self): self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(9), True) self.assertEqual(range_tracker.try_split((2, 8)), None) - self.assertEqual(range_tracker.try_split((2, 11)), ((2, 11), 11. / 12)) + self.assertEqual(range_tracker.try_split((2, 11)), + ((2, 11), old_div(11., 12))) self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(10), True) self.assertEqual(range_tracker.sub_range_tracker(2).try_claim(11), False) @@ -132,10 +133,10 @@ def test_estimate_size(self): def test_position_at_fration(self): ranges = [(0, 4), (4, 16), (16, 24), (24, 32)] - source = ConcatSource([iobase.SourceBundle((range[1] - range[0]) / 32., - RangeSource(*range), - None, None) - for range in ranges]) + source = ConcatSource([ + iobase.SourceBundle(old_div((range[1] - range[0]), 32.), + RangeSource(*range), None, None) + for range in ranges]) range_tracker = source.get_range_tracker() self.assertEquals(range_tracker.position_at_fraction(0), (0, 0)) @@ -177,10 +178,12 @@ def test_single_source(self): read_all = source_test_utils.read_from_source range10 = RangeSource(0, 10) - self.assertEquals(read_all(ConcatSource([range10])), range(10)) - self.assertEquals(read_all(ConcatSource([range10]), (0, 5)), range(5, 10)) + self.assertEquals(read_all(ConcatSource([range10])), + list(range(10))) + self.assertEquals(read_all(ConcatSource([range10]), (0, 5)), + list(range(5, 10))) self.assertEquals(read_all(ConcatSource([range10]), None, (0, 5)), - range(5)) + list(range(5))) def test_source_with_empty_ranges(self): read_all = source_test_utils.read_from_source @@ -190,11 +193,11 @@ def test_source_with_empty_ranges(self): range10 = RangeSource(0, 10) self.assertEquals(read_all(ConcatSource([empty, empty, range10])), - range(10)) + list(range(10))) self.assertEquals(read_all(ConcatSource([empty, range10, empty])), - range(10)) + list(range(10))) self.assertEquals(read_all(ConcatSource([range10, empty, range10, empty])), - range(10) + range(10)) + list(range(10)) + list(range(10))) def test_source_with_empty_ranges_exhastive(self): empty = RangeSource(0, 0) @@ -215,7 +218,7 @@ def test_run_concat_direct(self): ]) pipeline = TestPipeline() pcoll = pipeline | beam.io.Read(source) - assert_that(pcoll, equal_to(range(1000))) + assert_that(pcoll, equal_to(list(range(1000)))) pipeline.run() diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index eb99d0849017..6ebe756f8b72 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -24,16 +24,18 @@ import re import time import uuid +from builtins import range, zip + +from past.builtins import basestring from apache_beam.internal import util from apache_beam.io import iobase -from apache_beam.io.filesystem import BeamIOError -from apache_beam.io.filesystem import CompressionTypes +from apache_beam.io.filesystem import BeamIOError, CompressionTypes from apache_beam.io.filesystems import FileSystems +from apache_beam.options.value_provider import (StaticValueProvider, + ValueProvider, + check_accessible) from apache_beam.transforms.display import DisplayDataItem -from apache_beam.options.value_provider import ValueProvider -from apache_beam.options.value_provider import StaticValueProvider -from apache_beam.options.value_provider import check_accessible DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' @@ -197,12 +199,12 @@ def finalize_write(self, init_result, writer_results): source_files.append(shard) destination_files.append(final_name) - source_file_batch = [source_files[i:i + chunk_size] - for i in xrange(0, len(source_files), - chunk_size)] - destination_file_batch = [destination_files[i:i + chunk_size] - for i in xrange(0, len(destination_files), - chunk_size)] + source_file_batch = [ + source_files[i:i + chunk_size] + for i in range(0, len(source_files), chunk_size)] + destination_file_batch = [ + destination_files[i:i + chunk_size] + for i in range(0, len(destination_files), chunk_size)] logging.info( 'Starting finalize_write threads with num_shards: %d, ' @@ -221,7 +223,7 @@ def _rename_batch(batch): except BeamIOError as exp: if exp.exception_details is None: raise - for (src, dest), exception in exp.exception_details.iteritems(): + for (src, dest), exception in exp.exception_details.items(): if exception: logging.warning('Rename not successful: %s -> %s, %s', src, dest, exception) @@ -243,7 +245,7 @@ def _rename_batch(batch): return exceptions exception_batches = util.run_using_threadpool( - _rename_batch, zip(source_file_batch, destination_file_batch), + _rename_batch, list(zip(source_file_batch, destination_file_batch)), num_threads) all_exceptions = [e for exception_batch in exception_batches diff --git a/sdks/python/apache_beam/io/filebasedsink_test.py b/sdks/python/apache_beam/io/filebasedsink_test.py index 7c8ddb4072ef..d43e98dcb4c6 100644 --- a/sdks/python/apache_beam/io/filebasedsink_test.py +++ b/sdks/python/apache_beam/io/filebasedsink_test.py @@ -24,6 +24,7 @@ import shutil import tempfile import unittest +from builtins import range import hamcrest as hc import mock @@ -31,12 +32,11 @@ import apache_beam as beam from apache_beam.coders import coders from apache_beam.io import filebasedsink +from apache_beam.options.value_provider import StaticValueProvider from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher -from apache_beam.options.value_provider import StaticValueProvider - # TODO: Refactor code so all io tests are using same library # TestCaseWithTempDirCleanup class. diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py index 649693076c5f..52702be204c5 100644 --- a/sdks/python/apache_beam/io/filebasedsource.py +++ b/sdks/python/apache_beam/io/filebasedsource.py @@ -27,23 +27,19 @@ """ import uuid -from apache_beam.transforms.core import DoFn -from apache_beam.transforms.core import ParDo -from apache_beam.transforms.core import GroupByKey -from apache_beam.transforms.core import PTransform -from apache_beam.transforms.core import FlatMap -from apache_beam.transforms.core import Map +from past.builtins import basestring + from apache_beam.internal import pickler -from apache_beam.io import concat_source -from apache_beam.io import iobase -from apache_beam.io import range_trackers +from apache_beam.io import concat_source, iobase, range_trackers from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.filesystems import FileSystems from apache_beam.io.range_trackers import OffsetRange +from apache_beam.options.value_provider import (StaticValueProvider, + ValueProvider, + check_accessible) +from apache_beam.transforms.core import (DoFn, FlatMap, GroupByKey, Map, ParDo, + PTransform) from apache_beam.transforms.display import DisplayDataItem -from apache_beam.options.value_provider import ValueProvider -from apache_beam.options.value_provider import StaticValueProvider -from apache_beam.options.value_provider import check_accessible from apache_beam.transforms.trigger import DefaultTrigger MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 25 @@ -236,11 +232,11 @@ class _SingleFileSource(iobase.BoundedSource): def __init__(self, file_based_source, file_name, start_offset, stop_offset, min_bundle_size=0, splittable=True): - if not isinstance(start_offset, (int, long)): + if not isinstance(start_offset, int): raise TypeError( 'start_offset must be a number. Received: %r' % start_offset) if stop_offset != range_trackers.OffsetRangeTracker.OFFSET_INFINITY: - if not isinstance(stop_offset, (int, long)): + if not isinstance(stop_offset, int): raise TypeError( 'stop_offset must be a number. Received: %r' % stop_offset) if start_offset >= stop_offset: @@ -370,7 +366,7 @@ def expand(self, pvalue): return (keyed_pc | 'GroupByKey' >> GroupByKey() # Using FlatMap below due to the possibility of key collisions. - | 'DropKey' >> FlatMap(lambda (k, values): values)) + | 'DropKey' >> FlatMap(lambda k_values: k_values[1])) class _ReadRange(DoFn): diff --git a/sdks/python/apache_beam/io/filebasedsource_test.py b/sdks/python/apache_beam/io/filebasedsource_test.py index afb340d7608b..8c38be718ac5 100644 --- a/sdks/python/apache_beam/io/filebasedsource_test.py +++ b/sdks/python/apache_beam/io/filebasedsource_test.py @@ -1,51 +1,54 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# +from __future__ import division import bz2 -import cStringIO import gzip +import io import logging import math -import random import os +import random import tempfile import unittest +from builtins import object, range import hamcrest as hc +from future import standard_library +from past.utils import old_div import apache_beam as beam -from apache_beam.io import filebasedsource -from apache_beam.io import iobase -from apache_beam.io import range_trackers -from apache_beam.io.filesystem import CompressionTypes - +from apache_beam.io import filebasedsource, iobase, range_trackers # importing following private classes for testing from apache_beam.io.concat_source import ConcatSource -from apache_beam.io.filebasedsource import _SingleFileSource as SingleFileSource - +from apache_beam.io.filebasedsource import \ + _SingleFileSource as SingleFileSource from apache_beam.io.filebasedsource import FileBasedSource -from apache_beam.options.value_provider import StaticValueProvider -from apache_beam.options.value_provider import RuntimeValueProvider +from apache_beam.io.filesystem import CompressionTypes +from apache_beam.options.value_provider import (RuntimeValueProvider, + StaticValueProvider) from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +standard_library.install_aliases() + class LineSource(FileBasedSource): @@ -155,7 +158,7 @@ def __init__(self, values): def split(self, desired_bundle_size, start_position=None, stop_position=None): # simply devides values into two bundles - middle = len(self._values) / 2 + middle = old_div(len(self._values), 2) yield iobase.SourceBundle(0.5, TestConcatSource.DummySource( self._values[:middle]), None, None) yield iobase.SourceBundle(0.5, TestConcatSource.DummySource( @@ -185,16 +188,16 @@ def setUp(self): filebasedsource.MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 2 def test_read(self): - sources = [TestConcatSource.DummySource(range(start, start + 10)) for start - in [0, 10, 20]] + sources = [TestConcatSource.DummySource(list(range(start, start + 10))) + for start in [0, 10, 20]] concat = ConcatSource(sources) range_tracker = concat.get_range_tracker(None, None) read_data = [value for value in concat.read(range_tracker)] - self.assertItemsEqual(range(30), read_data) + self.assertItemsEqual(list(range(30)), read_data) def test_split(self): - sources = [TestConcatSource.DummySource(range(start, start + 10)) for start - in [0, 10, 20]] + sources = [TestConcatSource.DummySource(list(range(start, start + 10))) + for start in [0, 10, 20]] concat = ConcatSource(sources) splits = [split for split in concat.split()] self.assertEquals(6, len(splits)) @@ -207,11 +210,11 @@ def test_split(self): split.stop_position) read_data.extend([value for value in split.source.read( range_tracker_for_split)]) - self.assertItemsEqual(range(30), read_data) + self.assertItemsEqual(list(range(30)), read_data) def test_estimate_size(self): - sources = [TestConcatSource.DummySource(range(start, start + 10)) for start - in [0, 10, 20]] + sources = [TestConcatSource.DummySource(list(range(start, start + 10))) + for start in [0, 10, 20]] concat = ConcatSource(sources) self.assertEquals(30, concat.estimate_size()) @@ -332,7 +335,7 @@ def test_estimate_size_with_sampling_different_sizes(self): variance = 5 sizes = [] - for _ in xrange(num_files): + for _ in range(num_files): sizes.append(int(random.uniform(base_size - variance, base_size + variance))) pattern, _ = write_pattern(sizes) @@ -348,9 +351,9 @@ def test_splits_into_subranges(self): fbs = LineSource(pattern) splits = [split for split in fbs.split(desired_bundle_size=15)] expected_num_splits = ( - math.ceil(float(6 * 5) / 15) + - math.ceil(float(6 * 9) / 15) + - math.ceil(float(6 * 6) / 15)) + math.ceil(old_div(float(6 * 5), 15)) + + math.ceil(old_div(float(6 * 9), 15)) + + math.ceil(old_div(float(6 * 6), 15))) assert len(splits) == expected_num_splits def test_read_splits_single_file(self): @@ -454,7 +457,7 @@ def test_read_file_gzip(self): def test_read_pattern_bzip2(self): _, lines = write_data(200) splits = [0, 34, 100, 140, 164, 188, 200] - chunks = [lines[splits[i-1]:splits[i]] for i in xrange(1, len(splits))] + chunks = [lines[splits[i-1]:splits[i]] for i in range(1, len(splits))] compressed_chunks = [] for c in chunks: compressobj = bz2.BZ2Compressor() @@ -472,10 +475,10 @@ def test_read_pattern_bzip2(self): def test_read_pattern_gzip(self): _, lines = write_data(200) splits = [0, 34, 100, 140, 164, 188, 200] - chunks = [lines[splits[i-1]:splits[i]] for i in xrange(1, len(splits))] + chunks = [lines[splits[i-1]:splits[i]] for i in range(1, len(splits))] compressed_chunks = [] for c in chunks: - out = cStringIO.StringIO() + out = io.BytesIO() with gzip.GzipFile(fileobj=out, mode="w") as f: f.write('\n'.join(c)) compressed_chunks.append(out.getvalue()) @@ -519,10 +522,10 @@ def test_read_auto_single_file_gzip(self): def test_read_auto_pattern(self): _, lines = write_data(200) splits = [0, 34, 100, 140, 164, 188, 200] - chunks = [lines[splits[i - 1]:splits[i]] for i in xrange(1, len(splits))] + chunks = [lines[splits[i - 1]:splits[i]] for i in range(1, len(splits))] compressed_chunks = [] for c in chunks: - out = cStringIO.StringIO() + out = io.BytesIO() with gzip.GzipFile(fileobj=out, mode="w") as f: f.write('\n'.join(c)) compressed_chunks.append(out.getvalue()) @@ -538,11 +541,11 @@ def test_read_auto_pattern(self): def test_read_auto_pattern_compressed_and_uncompressed(self): _, lines = write_data(200) splits = [0, 34, 100, 140, 164, 188, 200] - chunks = [lines[splits[i - 1]:splits[i]] for i in xrange(1, len(splits))] + chunks = [lines[splits[i - 1]:splits[i]] for i in range(1, len(splits))] chunks_to_write = [] for i, c in enumerate(chunks): if i%2 == 0: - out = cStringIO.StringIO() + out = io.BytesIO() with gzip.GzipFile(fileobj=out, mode="w") as f: f.write('\n'.join(c)) chunks_to_write.append(out.getvalue()) diff --git a/sdks/python/apache_beam/io/filesystem.py b/sdks/python/apache_beam/io/filesystem.py index 5804d0048af0..96690179b17d 100644 --- a/sdks/python/apache_beam/io/filesystem.py +++ b/sdks/python/apache_beam/io/filesystem.py @@ -16,18 +16,32 @@ # """File system abstraction for file-based sources and sinks.""" -from __future__ import absolute_import +from __future__ import absolute_import, division import abc import bz2 -import cStringIO -import os -import zlib +import io import logging +import os +import sys import time +import zlib +from builtins import object + +from future import standard_library +from future.utils import with_metaclass +from past.builtins import basestring +from past.utils import old_div from apache_beam.utils.plugin import BeamPlugin +reload(sys) +if sys.version_info[0] < 3: + sys.setdefaultencoding('latin-1') + +standard_library.install_aliases() + + logger = logging.getLogger(__name__) DEFAULT_READ_BUFFER_SIZE = 16 * 1024 * 1024 @@ -80,7 +94,7 @@ def detect_compression_type(cls, file_path): """Returns the compression type of a file (based on its suffix).""" compression_types_by_suffix = {'.bz2': cls.BZIP2, '.gz': cls.GZIP} lowercased_path = file_path.lower() - for suffix, compression_type in compression_types_by_suffix.iteritems(): + for suffix, compression_type in compression_types_by_suffix.items(): if lowercased_path.endswith(suffix): return compression_type return cls.UNCOMPRESSED @@ -120,7 +134,7 @@ def __init__(self, if self.readable(): self._read_size = read_size - self._read_buffer = cStringIO.StringIO() + self._read_buffer = io.BytesIO() self._read_position = 0 self._read_eof = False @@ -235,20 +249,20 @@ def readline(self): if not self._decompressor: raise ValueError('decompressor not initialized') - io = cStringIO.StringIO() + stream = io.StringIO() while True: # Ensure that the internal buffer has at least half the read_size. Going # with half the _read_size (as opposed to a full _read_size) to ensure # that actual fetches are more evenly spread out, as opposed to having 2 # consecutive reads at the beginning of a read. - self._fetch_to_internal_buffer(self._read_size / 2) + self._fetch_to_internal_buffer(old_div(self._read_size, 2)) line = self._read_from_internal_buffer( lambda: self._read_buffer.readline()) - io.write(line) + stream.write(line.decode()) if line.endswith('\n') or not line: break # Newline or EOF reached. - return io.getvalue() + return stream.getvalue() def closed(self): return not self._file or self._file.closed() @@ -372,7 +386,7 @@ class FileMetadata(object): """ def __init__(self, path, size_in_bytes): assert isinstance(path, basestring) and path, "Path should be a string" - assert isinstance(size_in_bytes, (int, long)) and size_in_bytes >= 0, \ + assert isinstance(size_in_bytes, int) and size_in_bytes >= 0, \ "Invalid value for size_in_bytes should %s (of type %s)" % ( size_in_bytes, type(size_in_bytes)) self.path = path @@ -421,14 +435,13 @@ def __init__(self, msg, exception_details=None): self.exception_details = exception_details -class FileSystem(BeamPlugin): +class FileSystem(with_metaclass(abc.ABCMeta, BeamPlugin)): """A class that defines the functions that can be performed on a filesystem. All methods are abstract and they are for file system providers to implement. Clients should use the FileSystemUtil class to interact with the correct file system based on the provided file pattern scheme. """ - __metaclass__ = abc.ABCMeta CHUNK_SIZE = 1 # Chuck size in the batch operations @staticmethod diff --git a/sdks/python/apache_beam/io/filesystem_test.py b/sdks/python/apache_beam/io/filesystem_test.py index 607393d3a555..70ace6804e39 100644 --- a/sdks/python/apache_beam/io/filesystem_test.py +++ b/sdks/python/apache_beam/io/filesystem_test.py @@ -17,15 +17,28 @@ # """Unit tests for filesystem module.""" +from __future__ import division + import bz2 import gzip import os -import unittest +import sys import tempfile -from StringIO import StringIO +import unittest +from builtins import range + +from future import standard_library +from past.utils import old_div from apache_beam.io.filesystem import CompressedFile, CompressionTypes +standard_library.install_aliases() + +if sys.version_info[0] >= 3: + from io import StringIO +else: + from StringIO import StringIO + class TestCompressedFile(unittest.TestCase): """Base class for TestCases that deals with TempDir clean-up. @@ -126,10 +139,12 @@ def test_seek_cur(self): reference_fd = StringIO(self.content) # Test out of bound, inbound seeking in both directions - for seek_position in (-1, 0, 1, - len(self.content) / 2, - len(self.content) / 2, - -1 * len(self.content) / 2): + positions = [-1, 0, 1, + old_div(len(self.content), 2), + old_div(len(self.content), 2), + -1 * len(self.content) / 2] + positions = map(int, positions) + for seek_position in positions: compressed_fd.seek(seek_position, os.SEEK_CUR) reference_fd.seek(seek_position, os.SEEK_CUR) @@ -211,3 +226,7 @@ def test_tell(self): self.assertEqual(current_offset, readable.tell()) if not line: break + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py index e03968628d59..0a18f646d024 100644 --- a/sdks/python/apache_beam/io/filesystems.py +++ b/sdks/python/apache_beam/io/filesystems.py @@ -18,11 +18,9 @@ """FileSystems interface class for accessing the correct filesystem""" import re +from builtins import object -from apache_beam.io.filesystem import BeamIOError -from apache_beam.io.filesystem import CompressionTypes -from apache_beam.io.filesystem import FileSystem - +from apache_beam.io.filesystem import BeamIOError, CompressionTypes, FileSystem # All filesystem implements should be added here # pylint: disable=wrong-import-position, unused-import from apache_beam.io.localfilesystem import LocalFileSystem diff --git a/sdks/python/apache_beam/io/filesystems_test.py b/sdks/python/apache_beam/io/filesystems_test.py index 9a6f013ff479..4627c2270d98 100644 --- a/sdks/python/apache_beam/io/filesystems_test.py +++ b/sdks/python/apache_beam/io/filesystems_test.py @@ -18,12 +18,12 @@ """Unit tests for LocalFileSystem.""" -import unittest - import filecmp import os import shutil import tempfile +import unittest + import mock from apache_beam.io import localfilesystem @@ -124,7 +124,7 @@ def test_match_file_exception(self): FileSystems.match([None]) self.assertTrue( error.exception.message.startswith('Unable to get the Filesystem')) - self.assertEqual(error.exception.exception_details.keys(), [None]) + self.assertEqual(list(error.exception.exception_details.keys()), [None]) def test_match_directory(self): path1 = os.path.join(self.tmpdir, 'f1') @@ -159,7 +159,8 @@ def test_copy_error(self): FileSystems.copy([path1], [path2]) self.assertTrue( error.exception.message.startswith('Copy operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [(path1, path2)]) + self.assertEqual(list(error.exception.exception_details.keys()), + [(path1, path2)]) def test_copy_directory(self): path_t1 = os.path.join(self.tmpdir, 't1') @@ -192,7 +193,8 @@ def test_rename_error(self): FileSystems.rename([path1], [path2]) self.assertTrue( error.exception.message.startswith('Rename operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [(path1, path2)]) + self.assertEqual(list(error.exception.exception_details.keys()), + [(path1, path2)]) def test_rename_directory(self): path_t1 = os.path.join(self.tmpdir, 't1') @@ -234,4 +236,4 @@ def test_delete_error(self): FileSystems.delete([path1]) self.assertTrue( error.exception.message.startswith('Delete operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [path1]) + self.assertEqual(list(error.exception.exception_details.keys()), [path1]) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 33d67bf7c53a..e817dd784c8e 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -109,19 +109,19 @@ import re import time import uuid +from builtins import object, zip + +from past.builtins import basestring from apache_beam import coders from apache_beam.internal.gcp import auth -from apache_beam.internal.gcp.json_value import from_json_value -from apache_beam.internal.gcp.json_value import to_json_value +from apache_beam.internal.gcp.json_value import from_json_value, to_json_value +from apache_beam.io.gcp.internal.clients import bigquery +from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.runners.dataflow.native_io import iobase as dataflow_io -from apache_beam.transforms import DoFn -from apache_beam.transforms import ParDo -from apache_beam.transforms import PTransform +from apache_beam.transforms import DoFn, ParDo, PTransform from apache_beam.transforms.display import DisplayDataItem from apache_beam.utils import retry -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.io.gcp.internal.clients import bigquery # Protect against environments where bigquery library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -190,8 +190,8 @@ def encode(self, table_row): try: return json.dumps( collections.OrderedDict( - zip(self.field_names, - [from_json_value(f.v) for f in table_row.f])), + list(zip(self.field_names, + [from_json_value(f.v) for f in table_row.f]))), allow_nan=False) except ValueError as e: raise ValueError('%s. %s' % (e, JSON_COMPLIANCE_ERROR)) @@ -200,7 +200,7 @@ def decode(self, encoded_table_row): od = json.loads( encoded_table_row, object_pairs_hook=collections.OrderedDict) return bigquery.TableRow( - f=[bigquery.TableCell(v=to_json_value(e)) for e in od.itervalues()]) + f=[bigquery.TableCell(v=to_json_value(e)) for e in od.values()]) def parse_table_schema_from_json(schema_string): @@ -1091,7 +1091,7 @@ def insert_rows(self, project_id, dataset_id, table_id, rows): final_rows = [] for row in rows: json_object = bigquery.JsonObject() - for k, v in row.iteritems(): + for k, v in row.items(): json_object.additionalProperties.append( bigquery.JsonObject.AdditionalProperty( key=k, value=to_json_value(v))) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index bfd06ace6400..9a3d00522898 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -27,14 +27,13 @@ import mock import apache_beam as beam -from apache_beam.io.gcp.bigquery import RowAsDictJsonCoder -from apache_beam.io.gcp.bigquery import TableRowJsonCoder -from apache_beam.io.gcp.bigquery import parse_table_schema_from_json -from apache_beam.io.gcp.internal.clients import bigquery from apache_beam.internal.gcp.json_value import to_json_value +from apache_beam.io.gcp.bigquery import (RowAsDictJsonCoder, TableRowJsonCoder, + parse_table_schema_from_json) +from apache_beam.io.gcp.internal.clients import bigquery +from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher -from apache_beam.options.pipeline_options import PipelineOptions # Protect against environments where bigquery library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -706,7 +705,7 @@ def test_rows_are_written(self): sample_row = {'i': 1, 'b': True, 's': 'abc', 'f': 3.14} expected_rows = [] json_object = bigquery.JsonObject() - for k, v in sample_row.iteritems(): + for k, v in sample_row.items(): json_object.additionalProperties.append( bigquery.JsonObject.AdditionalProperty( key=k, value=to_json_value(v))) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py index 7d94f24ca859..d547f61237cb 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py @@ -1,3 +1,12 @@ +from __future__ import division + +import random +from builtins import object + +from past.utils import old_div + +from apache_beam.io.gcp.datastore.v1 import util + # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -19,10 +28,6 @@ # # For internal use only; no backwards-compatibility guarantees. -import random - -from apache_beam.io.gcp.datastore.v1 import util - class AdaptiveThrottler(object): """Implements adaptive throttling. @@ -63,8 +68,9 @@ def _throttling_probability(self, now): all_requests = self._all_requests.sum(now) successful_requests = self._successful_requests.sum(now) return max( - 0, (all_requests - self._overload_ratio * successful_requests) - / (all_requests + AdaptiveThrottler.MIN_REQUESTS)) + 0, + old_div((all_requests - self._overload_ratio * successful_requests), + (all_requests + AdaptiveThrottler.MIN_REQUESTS))) def throttle_request(self, now): """Determines whether one RPC attempt should be throttled. diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py index 93b91ad7cfab..c268dcae29ce 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py @@ -1,3 +1,14 @@ +from __future__ import division + +import unittest +from builtins import range + +from mock import patch +from past.utils import old_div + +from apache_beam.io.gcp.datastore.v1.adaptive_throttler import \ + AdaptiveThrottler + # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -15,11 +26,6 @@ # limitations under the License. # -import unittest -from mock import patch - -from apache_beam.io.gcp.datastore.v1.adaptive_throttler import AdaptiveThrottler - class AdaptiveThrottlerTest(unittest.TestCase): @@ -70,7 +76,8 @@ def test_no_throttling_after_errors_expire(self): @patch('random.Random') def test_throttling_after_errors(self, mock_random): - mock_random().uniform.side_effect = [x/10.0 for x in range(0, 10)]*2 + mock_random().uniform.side_effect = [old_div(x, 10.0) + for x in range(0, 10)]*2 self._throttler = AdaptiveThrottler( AdaptiveThrottlerTest.SAMPLE_PERIOD, AdaptiveThrottlerTest.BUCKET, AdaptiveThrottlerTest.OVERLOAD_RATIO) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py index 3cfba93d2e8c..05450cac1ba3 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py @@ -16,9 +16,21 @@ # """A connector for reading from and writing to Google Cloud Datastore""" +from __future__ import division import logging import time +from builtins import object + +from past.utils import old_div + +from apache_beam.io.gcp.datastore.v1 import helper, query_splitter, util +from apache_beam.io.gcp.datastore.v1.adaptive_throttler import \ + AdaptiveThrottler +from apache_beam.metrics.metric import Metrics +from apache_beam.transforms import (Create, DoFn, FlatMap, GroupByKey, Map, + ParDo, PTransform) +from apache_beam.transforms.util import Values # Protect against environments where datastore library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -29,19 +41,6 @@ pass # pylint: enable=wrong-import-order, wrong-import-position -from apache_beam.io.gcp.datastore.v1 import helper -from apache_beam.io.gcp.datastore.v1 import query_splitter -from apache_beam.io.gcp.datastore.v1 import util -from apache_beam.io.gcp.datastore.v1.adaptive_throttler import AdaptiveThrottler -from apache_beam.transforms import Create -from apache_beam.transforms import DoFn -from apache_beam.transforms import FlatMap -from apache_beam.transforms import GroupByKey -from apache_beam.transforms import Map -from apache_beam.transforms import PTransform -from apache_beam.transforms import ParDo -from apache_beam.transforms.util import Values -from apache_beam.metrics.metric import Metrics __all__ = ['ReadFromDatastore', 'WriteToDatastore', 'DeleteFromDatastore'] @@ -299,8 +298,8 @@ def get_estimated_num_splits(project, namespace, query, datastore): project, namespace, query, datastore) logging.info('Estimated size bytes for query: %s', estimated_size_bytes) num_splits = int(min(ReadFromDatastore._NUM_QUERY_SPLITS_MAX, round( - (float(estimated_size_bytes) / - ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES)))) + (old_div(float(estimated_size_bytes), + ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES))))) except Exception as e: logging.warning('Failed to fetch estimated size bytes: %s', e) @@ -360,12 +359,13 @@ def get_batch_size(self, now): if not self._commit_time_per_entity_ms.has_data(now): return _Mutate._WRITE_BATCH_INITIAL_SIZE - recent_mean_latency_ms = (self._commit_time_per_entity_ms.sum(now) - / self._commit_time_per_entity_ms.count(now)) + recent_mean_latency_ms = ( + old_div(self._commit_time_per_entity_ms.sum(now), + self._commit_time_per_entity_ms.count(now))) return max(_Mutate._WRITE_BATCH_MIN_SIZE, min(_Mutate._WRITE_BATCH_MAX_SIZE, - _Mutate._WRITE_BATCH_TARGET_LATENCY_MS - / max(recent_mean_latency_ms, 1) + old_div(_Mutate._WRITE_BATCH_TARGET_LATENCY_MS, + max(recent_mean_latency_ms, 1)) )) def report_latency(self, now, latency_ms, num_mutations): @@ -376,7 +376,8 @@ def report_latency(self, now, latency_ms, num_mutations): latency_ms: double, the observed latency in milliseconds for this RPC. num_mutations: int, number of mutations contained in the RPC. """ - self._commit_time_per_entity_ms.add(now, latency_ms / num_mutations) + self._commit_time_per_entity_ms.add(now, + old_div(latency_ms, num_mutations)) class DatastoreWriteFn(DoFn): """A ``DoFn`` that write mutations to Datastore. @@ -443,7 +444,7 @@ def _flush_batch(self): _, latency_ms = helper.write_mutations( self._datastore, self._project, self._mutations, self._throttler, self._update_rpc_stats, - throttle_delay=_Mutate._WRITE_BATCH_TARGET_LATENCY_MS/1000) + throttle_delay=old_div(_Mutate._WRITE_BATCH_TARGET_LATENCY_MS, 1000)) logging.debug("Successfully wrote %d mutations in %dms.", len(self._mutations), latency_ms) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py index 72c4c8cc0c13..8c025b190cd1 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py @@ -1,3 +1,17 @@ +from __future__ import division, print_function + +import unittest +from builtins import map, range, zip + +from mock import MagicMock, call, patch +from past.utils import old_div + +from apache_beam.io.gcp.datastore.v1 import (fake_datastore, helper, + query_splitter) +from apache_beam.io.gcp.datastore.v1.datastoreio import (ReadFromDatastore, + WriteToDatastore, + _Mutate) + # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -15,16 +29,6 @@ # limitations under the License. # -import unittest - -from mock import MagicMock, call, patch - -from apache_beam.io.gcp.datastore.v1 import fake_datastore -from apache_beam.io.gcp.datastore.v1 import helper -from apache_beam.io.gcp.datastore.v1 import query_splitter -from apache_beam.io.gcp.datastore.v1.datastoreio import _Mutate -from apache_beam.io.gcp.datastore.v1.datastoreio import ReadFromDatastore -from apache_beam.io.gcp.datastore.v1.datastoreio import WriteToDatastore # Protect against environments where datastore library is not available. # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports @@ -174,7 +178,8 @@ def check_DatastoreWriteFn(self, num_entities): entities = [e.entity for e in fake_datastore.create_entities(num_entities)] - expected_mutations = map(WriteToDatastore.to_upsert_mutation, entities) + expected_mutations = list( + map(WriteToDatastore.to_upsert_mutation, entities)) actual_mutations = [] self._mock_datastore.commit.side_effect = ( @@ -190,7 +195,7 @@ def check_DatastoreWriteFn(self, num_entities): self.assertEqual(actual_mutations, expected_mutations) self.assertEqual( - (num_entities - 1) / _Mutate._WRITE_BATCH_INITIAL_SIZE + 1, + old_div((num_entities - 1), _Mutate._WRITE_BATCH_INITIAL_SIZE) + 1, self._mock_datastore.commit.call_count) def test_DatastoreWriteLargeEntities(self): @@ -212,7 +217,7 @@ def test_DatastoreWriteLargeEntities(self): def verify_unique_keys(self, queries): """A helper function that verifies if all the queries have unique keys.""" - keys, _ = zip(*queries) + keys, _ = list(zip(*queries)) keys = set(keys) self.assertEqual(len(keys), len(queries)) @@ -236,7 +241,7 @@ def fake_run_query(req): elif req == kind_stat_req: return kind_stat_resp else: - print kind_stat_req + print(kind_stat_req) raise ValueError("Unknown req: %s" % req) self._mock_datastore.run_query.side_effect = fake_run_query diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/fake_datastore.py b/sdks/python/apache_beam/io/gcp/datastore/v1/fake_datastore.py index aa3780558d7f..b9f546bccfe9 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/fake_datastore.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/fake_datastore.py @@ -21,6 +21,7 @@ """ import uuid +from builtins import range # Protect against environments where datastore library is not available. # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py index 5cde25540da4..fe5b41e4c64b 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py @@ -21,13 +21,19 @@ """ import errno -from socket import error as SocketError import logging import sys import time +from builtins import next, object +from socket import error as SocketError + +from past.builtins import cmp + +from apache_beam.internal.gcp import auth +from apache_beam.utils import retry # Protect against environments where datastore library is not available. -# pylint: disable=wrong-import-order, wrong-import-position +# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: from google.cloud.proto.datastore.v1 import datastore_pb2 from google.cloud.proto.datastore.v1 import entity_pb2 @@ -39,10 +45,7 @@ from googledatastore.connection import RPCError except ImportError: pass -# pylint: enable=wrong-import-order, wrong-import-position - -from apache_beam.internal.gcp import auth -from apache_beam.utils import retry +# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports def key_comparator(k1, k2): @@ -249,7 +252,7 @@ def make_kind_stats_query(namespace, kind, latest_timestamp): kind_stat_query.kind.add().name = '__Stat_Ns_Kind__' kind_filter = datastore_helper.set_property_filter( - query_pb2.Filter(), 'kind_name', PropertyFilter.EQUAL, unicode(kind)) + query_pb2.Filter(), 'kind_name', PropertyFilter.EQUAL, str(kind)) timestamp_filter = datastore_helper.set_property_filter( query_pb2.Filter(), 'timestamp', PropertyFilter.EQUAL, latest_timestamp) @@ -274,7 +277,7 @@ def __init__(self, project, namespace, query, datastore): self._project = project self._namespace = namespace self._start_cursor = None - self._limit = self._query.limit.value or sys.maxint + self._limit = self._query.limit.value or sys.maxsize self._req = make_request(project, namespace, query) @retry.with_exponential_backoff(num_retries=5, diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/helper_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/helper_test.py index a8b1bb11231f..43b6d0c50031 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/helper_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/helper_test.py @@ -18,19 +18,18 @@ """Tests for datastore helper.""" import errno import random -from socket import error as SocketError import sys import unittest +from builtins import map +from socket import error as SocketError from mock import MagicMock -from apache_beam.io.gcp.datastore.v1 import fake_datastore -from apache_beam.io.gcp.datastore.v1 import helper +from apache_beam.io.gcp.datastore.v1 import fake_datastore, helper from apache_beam.testing.test_utils import patch_retry - # Protect against environments where apitools library is not available. -# pylint: disable=wrong-import-order, wrong-import-position +# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: from google.cloud.proto.datastore.v1 import datastore_pb2 from google.cloud.proto.datastore.v1 import entity_pb2 @@ -41,7 +40,7 @@ from googledatastore import helper as datastore_helper except ImportError: datastore_helper = None -# pylint: enable=wrong-import-order, wrong-import-position +# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports @unittest.skipIf(datastore_helper is None, 'GCP dependencies are not installed') @@ -153,7 +152,7 @@ def check_query_iterator(self, num_entities, batch_size, query): self.assertEqual(entity, entities[i].entity) i += 1 - limit = query.limit.value if query.HasField('limit') else sys.maxint + limit = query.limit.value if query.HasField('limit') else sys.maxsize self.assertEqual(i, min(num_entities, limit)) def test_is_key_valid(self): diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter.py b/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter.py index d5674f9cbf15..1c605d12e8f6 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter.py @@ -16,6 +16,11 @@ # """Implements a Cloud Datastore query splitter.""" +from __future__ import division + +from builtins import range + +from past.utils import old_div from apache_beam.io.gcp.datastore.v1 import helper @@ -225,7 +230,7 @@ def _get_split_key(keys, num_splits): # distributed throughout the splits and don't make the last split # significantly larger than the rest. - num_keys_per_split = max(1.0, float(len(keys)) / (num_splits - 1)) + num_keys_per_split = max(1.0, old_div(float(len(keys)), (num_splits - 1))) split_keys = [] diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter_test.py index 52f25facd058..80ca2277c740 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/query_splitter_test.py @@ -19,11 +19,9 @@ import unittest -from mock import MagicMock -from mock import call +from mock import MagicMock, call -from apache_beam.io.gcp.datastore.v1 import fake_datastore -from apache_beam.io.gcp.datastore.v1 import query_splitter +from apache_beam.io.gcp.datastore.v1 import fake_datastore, query_splitter # Protect against environments where datastore library is not available. # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/util.py b/sdks/python/apache_beam/io/gcp/datastore/v1/util.py index 5670a241ba80..4579d0bc68d4 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/util.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/util.py @@ -1,3 +1,10 @@ +from __future__ import division + +import math +from builtins import object, range + +from past.utils import old_div + # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -19,8 +26,6 @@ # # For internal use only; no backwards-compatibility guarantees. -import math - class MovingSum(object): """Class that keeps track of a rolling window sum. @@ -36,14 +41,14 @@ class MovingSum(object): def __init__(self, window_ms, bucket_ms): if window_ms <= bucket_ms or bucket_ms <= 0: raise ValueError("window_ms > bucket_ms > 0 please") - self._num_buckets = int(math.ceil(window_ms / bucket_ms)) + self._num_buckets = int(math.ceil(old_div(window_ms, bucket_ms))) self._bucket_ms = bucket_ms self._Reset(now=0) # initialize the moving window members def _Reset(self, now): self._current_index = 0 # pointer into self._buckets self._current_ms_since_epoch = math.floor( - now / self._bucket_ms) * self._bucket_ms + old_div(now, self._bucket_ms)) * self._bucket_ms # _buckets is a list where each element is a list [sum, num_samples] # This is a circular buffer where diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py index ce8b5e6e4244..0ac72b59ad43 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py @@ -18,12 +18,11 @@ from __future__ import absolute_import -from apache_beam.io.filesystem import BeamIOError -from apache_beam.io.filesystem import CompressedFile -from apache_beam.io.filesystem import CompressionTypes -from apache_beam.io.filesystem import FileMetadata -from apache_beam.io.filesystem import FileSystem -from apache_beam.io.filesystem import MatchResult +from builtins import zip + +from apache_beam.io.filesystem import (BeamIOError, CompressedFile, + CompressionTypes, FileMetadata, + FileSystem, MatchResult) from apache_beam.io.gcp import gcsio __all__ = ['GCSFileSystem'] @@ -123,7 +122,7 @@ def _match(pattern, limit): pattern += '*' file_sizes = gcsio.GcsIO().size_of_files_in_glob(pattern, limit) metadata_list = [FileMetadata(path, size) - for path, size in file_sizes.iteritems()] + for path, size in file_sizes.items()] return MatchResult(pattern, metadata_list) exceptions = {} diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py index 923fc7dd4b68..a17cd50e7cbd 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py @@ -19,10 +19,11 @@ """Unit tests for GCS File System.""" import unittest +from builtins import zip import mock -from apache_beam.io.filesystem import BeamIOError -from apache_beam.io.filesystem import FileMetadata + +from apache_beam.io.filesystem import BeamIOError, FileMetadata # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index b18a2875aa0c..9c3b9dc98f9b 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -20,21 +20,27 @@ https://github.com/GoogleCloudPlatform/appengine-gcs-client. """ -import cStringIO import errno import fnmatch import logging import multiprocessing import os -import Queue +import queue import re import threading import time import traceback +from builtins import object + import httplib2 +import six +from future import standard_library from apache_beam.utils import retry +standard_library.install_aliases() + + __all__ = ['GcsIO'] @@ -452,7 +458,7 @@ def __init__(self, self.get_request.generation = metadata.generation # Initialize read buffer state. - self.download_stream = cStringIO.StringIO() + self.download_stream = six.BytesIO() self.downloader = transfer.Download( self.download_stream, auto_transfer=False, chunksize=self.buffer_size) self.client.objects.Get(self.get_request, download=self.downloader) @@ -469,12 +475,12 @@ def _get_object_metadata(self, get_request): def __iter__(self): return self - def __next__(self): + def next(self): """Read one line delimited by '\\n' from the file. """ - return self.next() + return self.__next__() - def next(self): + def __next__(self): """Read one line delimited by '\\n' from the file. """ line = self.readline() @@ -575,22 +581,22 @@ def _fetch_next_if_buffer_exhausted(self): self.buffer_start_position = self.position retry_count = 0 while retry_count <= 10: - queue = Queue.Queue() + myqueue = queue.Queue() t = threading.Thread(target=self._fetch_to_queue, - args=(queue, self._get_segment, + args=(myqueue, self._get_segment, (self.position, bytes_to_request))) t.daemon = True t.start() try: - result, exn, tb = queue.get(timeout=self.segment_timeout) - except Queue.Empty: + result, exn, tb = myqueue.get(timeout=self.segment_timeout) + except queue.Empty: logging.warning( ('Timed out fetching %d bytes from position %d of %s after %f ' 'seconds; retrying...'), bytes_to_request, self.position, self.path, self.segment_timeout) retry_count += 1 # Reinitialize download objects. - self.download_stream = cStringIO.StringIO() + self.download_stream = six.BytesIO() self.downloader = transfer.Download( self.download_stream, auto_transfer=False, chunksize=self.buffer_size) @@ -607,13 +613,13 @@ def _fetch_next_if_buffer_exhausted(self): raise GcsIOError( 'Reached retry limit for _fetch_next_if_buffer_exhausted.') - def _fetch_to_queue(self, queue, func, args): + def _fetch_to_queue(self, myqueue, func, args): try: value = func(*args) - queue.put((value, None, None)) + myqueue.put((value, None, None)) except Exception as e: # pylint: disable=broad-except tb = traceback.format_exc() - queue.put((None, e, tb)) + myqueue.put((None, e, tb)) def _remaining(self): return self.size - self.position @@ -636,7 +642,7 @@ def _get_segment(self, start, size): end = start + size - 1 downloader.GetRange(start, end) value = download_stream.getvalue() - # Clear the cStringIO object after we've read its contents. + # Clear the IO object after we've read its contents. download_stream.truncate(0) assert len(value) == size return value diff --git a/sdks/python/apache_beam/io/gcp/gcsio_test.py b/sdks/python/apache_beam/io/gcp/gcsio_test.py index 06a82272900e..714b689473dd 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_test.py @@ -15,6 +15,7 @@ # limitations under the License. # """Tests for Google Cloud Storage client.""" +from __future__ import division import errno import logging @@ -24,9 +25,11 @@ import threading import time import unittest +from builtins import object, range import httplib2 import mock +from past.utils import old_div # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -117,7 +120,7 @@ def Insert(self, insert_request, upload=None): # pylint: disable=invalid-name if not data: break data_list.append(data) - f.contents = ''.join(data_list) + f.contents = ''.join(data_list).decode('latin-1').encode('latin-1') self.add_file(f) @@ -225,7 +228,8 @@ class TestGCSIO(unittest.TestCase): def _insert_random_file(self, client, path, size, generation=1): bucket, name = gcsio.parse_gcs_path(path) - f = FakeFile(bucket, name, os.urandom(size), generation) + random_contents = os.urandom(size).decode('latin-1').encode('latin-1') + f = FakeFile(bucket, name, random_contents, generation) client.objects.add_file(f) return f @@ -486,8 +490,10 @@ def test_file_random_seek(self): start, end = min(a, b), max(a, b) f.seek(start) self.assertEqual(f.tell(), start) - self.assertEqual( - f.read(end - start + 1), random_file.contents[start:end + 1]) + read_result = f.read(end - start + 1) + file_contents = random_file.contents[start:end + 1] + self.assertEqual(type(read_result), type(file_contents)) + self.assertEqual(read_result, file_contents) self.assertEqual(f.tell(), end + 1) def test_file_iterator(self): @@ -499,7 +505,7 @@ def test_file_iterator(self): line = os.urandom(line_length).replace('\n', ' ') + '\n' lines.append(line) - contents = ''.join(lines) + contents = ''.join(lines).decode('latin-1').encode('latin-1') bucket, name = gcsio.parse_gcs_path(file_name) self.client.objects.add_file(FakeFile(bucket, name, contents, 1)) @@ -625,7 +631,7 @@ def test_context_manager(self): # Test that exceptions are not swallowed by the context manager. with self.assertRaises(ZeroDivisionError): with self.gcs.open(file_name) as f: - f.read(0 / 0) + f.read(old_div(0, 0)) def test_glob(self): bucket_name = 'gcsio-test' diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_client.py b/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_client.py index 201a1830b878..b32fc5638f7d 100644 --- a/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_client.py +++ b/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_client.py @@ -19,7 +19,8 @@ # NOTE: This file is autogenerated and should not be edited by hand. from apitools.base.py import base_api -from apache_beam.io.gcp.internal.clients.bigquery import bigquery_v2_messages as messages +from apache_beam.io.gcp.internal.clients.bigquery import \ + bigquery_v2_messages as messages class BigqueryV2(base_api.BaseApiClient): diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_messages.py b/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_messages.py index 404542863f3c..4f3bae3d6abe 100644 --- a/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_messages.py +++ b/sdks/python/apache_beam/io/gcp/internal/clients/bigquery/bigquery_v2_messages.py @@ -22,9 +22,7 @@ # NOTE: This file is autogenerated and should not be edited by hand. from apitools.base.protorpclite import messages as _messages -from apitools.base.py import encoding -from apitools.base.py import extra_types - +from apitools.base.py import encoding, extra_types package = 'bigquery' @@ -1906,5 +1904,3 @@ class ViewDefinition(_messages.Message): query = _messages.StringField(1) userDefinedFunctionResources = _messages.MessageField('UserDefinedFunctionResource', 2, repeated=True) - - diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py index 1b46d917f143..72b453dab19c 100644 --- a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py +++ b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py @@ -19,7 +19,8 @@ # NOTE: This file is autogenerated and should not be edited by hand. from apitools.base.py import base_api -from apache_beam.io.gcp.internal.clients.storage import storage_v1_messages as messages +from apache_beam.io.gcp.internal.clients.storage import \ + storage_v1_messages as messages class StorageV1(base_api.BaseApiClient): diff --git a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py index dc9e5e692554..d1d226881adf 100644 --- a/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py +++ b/sdks/python/apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py @@ -23,9 +23,7 @@ from apitools.base.protorpclite import message_types as _message_types from apitools.base.protorpclite import messages as _messages -from apitools.base.py import encoding -from apitools.base.py import extra_types - +from apitools.base.py import encoding, extra_types package = 'storage' @@ -1916,5 +1914,3 @@ class ProjectionValueValuesEnum(_messages.Enum): prefix = _messages.StringField(6) projection = _messages.EnumField('ProjectionValueValuesEnum', 7) versions = _messages.BooleanField(8) - - diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 7d1f3553f4fa..8e1673a8f23f 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -25,17 +25,17 @@ from __future__ import absolute_import import re +import sys from apache_beam import coders -from apache_beam.io.iobase import Read -from apache_beam.io.iobase import Write +from apache_beam.io.iobase import Read, Write from apache_beam.runners.dataflow.native_io import iobase as dataflow_io -from apache_beam.transforms import core -from apache_beam.transforms import PTransform -from apache_beam.transforms import Map -from apache_beam.transforms import window +from apache_beam.transforms import Map, PTransform, core, window from apache_beam.transforms.display import DisplayDataItem +if sys.version_info[0] >= 3: + unicode = str + __all__ = ['ReadStringsFromPubSub', 'WriteStringsToPubSub'] diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py index 0dcc3c39ab5f..ee67bae3058b 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py @@ -22,15 +22,14 @@ import hamcrest as hc -from apache_beam.io.gcp.pubsub import _PubSubPayloadSink -from apache_beam.io.gcp.pubsub import _PubSubPayloadSource -from apache_beam.io.gcp.pubsub import ReadStringsFromPubSub -from apache_beam.io.gcp.pubsub import WriteStringsToPubSub +from apache_beam.io.gcp.pubsub import (ReadStringsFromPubSub, + WriteStringsToPubSub, + _PubSubPayloadSink, + _PubSubPayloadSource) from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher - # Protect against environments where the PubSub library is not available. # pylint: disable=wrong-import-order, wrong-import-position try: @@ -47,7 +46,7 @@ def test_expand_with_topic(self): pcoll = p | ReadStringsFromPubSub('projects/fakeprj/topics/a_topic', None, 'a_label') # Ensure that the output type is str - self.assertEqual(unicode, pcoll.element_type) + self.assertEqual(str, pcoll.element_type) # Ensure that the properties passed through correctly source = pcoll.producer.transform._source @@ -59,7 +58,7 @@ def test_expand_with_subscription(self): pcoll = p | ReadStringsFromPubSub( None, 'projects/fakeprj/subscriptions/a_subscription', 'a_label') # Ensure that the output type is str - self.assertEqual(unicode, pcoll.element_type) + self.assertEqual(str, pcoll.element_type) # Ensure that the properties passed through correctly source = pcoll.producer.transform._source diff --git a/sdks/python/apache_beam/io/gcp/tests/utils_test.py b/sdks/python/apache_beam/io/gcp/tests/utils_test.py index 270750a25217..702c43b4955d 100644 --- a/sdks/python/apache_beam/io/gcp/tests/utils_test.py +++ b/sdks/python/apache_beam/io/gcp/tests/utils_test.py @@ -19,6 +19,7 @@ import logging import unittest + from mock import Mock, patch from apache_beam.io.gcp.tests import utils diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index db75fe38baa9..deef11756205 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -29,22 +29,17 @@ the sink. """ -from collections import namedtuple - import logging import random import uuid +from builtins import object, range +from collections import namedtuple -from apache_beam import pvalue -from apache_beam import coders +from apache_beam import coders, pvalue from apache_beam.portability.api import beam_runner_api_pb2 -from apache_beam.pvalue import AsIter -from apache_beam.pvalue import AsSingleton -from apache_beam.transforms import core -from apache_beam.transforms import ptransform -from apache_beam.transforms import window -from apache_beam.transforms.display import HasDisplayData -from apache_beam.transforms.display import DisplayDataItem +from apache_beam.pvalue import AsIter, AsSingleton +from apache_beam.transforms import core, ptransform, window +from apache_beam.transforms.display import DisplayDataItem, HasDisplayData from apache_beam.utils import urns from apache_beam.utils.windowed_value import WindowedValue diff --git a/sdks/python/apache_beam/io/localfilesystem.py b/sdks/python/apache_beam/io/localfilesystem.py index b08ac494846e..c7fb7e4aa97e 100644 --- a/sdks/python/apache_beam/io/localfilesystem.py +++ b/sdks/python/apache_beam/io/localfilesystem.py @@ -19,15 +19,14 @@ from __future__ import absolute_import import glob +import io import os import shutil +from builtins import zip -from apache_beam.io.filesystem import BeamIOError -from apache_beam.io.filesystem import CompressedFile -from apache_beam.io.filesystem import CompressionTypes -from apache_beam.io.filesystem import FileMetadata -from apache_beam.io.filesystem import FileSystem -from apache_beam.io.filesystem import MatchResult +from apache_beam.io.filesystem import (BeamIOError, CompressedFile, + CompressionTypes, FileMetadata, + FileSystem, MatchResult) __all__ = ['LocalFileSystem'] @@ -121,7 +120,7 @@ def _path_open(self, path, mode, mime_type='application/octet-stream', """Helper functions to open a file in the provided mode. """ compression_type = FileSystem._get_compression_type(path, compression_type) - raw_file = open(path, mode) + raw_file = io.open(path, mode) if compression_type == CompressionTypes.UNCOMPRESSED: return raw_file else: diff --git a/sdks/python/apache_beam/io/localfilesystem_test.py b/sdks/python/apache_beam/io/localfilesystem_test.py index 04cf5b7c6f4d..8d97d60377ef 100644 --- a/sdks/python/apache_beam/io/localfilesystem_test.py +++ b/sdks/python/apache_beam/io/localfilesystem_test.py @@ -18,12 +18,12 @@ """Unit tests for LocalFileSystem.""" -import unittest - import filecmp import os import shutil import tempfile +import unittest + import mock from apache_beam.io import localfilesystem @@ -144,7 +144,7 @@ def test_match_file_exception(self): self.fs.match([None]) self.assertTrue( error.exception.message.startswith('Match operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [None]) + self.assertEqual(list(error.exception.exception_details.keys()), [None]) def test_match_directory(self): path1 = os.path.join(self.tmpdir, 'f1') @@ -179,7 +179,8 @@ def test_copy_error(self): self.fs.copy([path1], [path2]) self.assertTrue( error.exception.message.startswith('Copy operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [(path1, path2)]) + self.assertEqual(list(error.exception.exception_details.keys()), + [(path1, path2)]) def test_copy_directory(self): path_t1 = os.path.join(self.tmpdir, 't1') @@ -212,7 +213,8 @@ def test_rename_error(self): self.fs.rename([path1], [path2]) self.assertTrue( error.exception.message.startswith('Rename operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [(path1, path2)]) + self.assertEqual(list(error.exception.exception_details.keys()), + [(path1, path2)]) def test_rename_directory(self): path_t1 = os.path.join(self.tmpdir, 't1') @@ -254,4 +256,4 @@ def test_delete_error(self): self.fs.delete([path1]) self.assertTrue( error.exception.message.startswith('Delete operation failed')) - self.assertEqual(error.exception.exception_details.keys(), [path1]) + self.assertEqual(list(error.exception.exception_details.keys()), [path1]) diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py index 1339b911efc5..66a3b510fe1d 100644 --- a/sdks/python/apache_beam/io/range_trackers.py +++ b/sdks/python/apache_beam/io/range_trackers.py @@ -17,10 +17,14 @@ """iobase.RangeTracker implementations provided with Dataflow SDK. """ +from __future__ import division import logging import math import threading +from builtins import object, zip + +from past.utils import old_div from apache_beam.io import iobase @@ -59,7 +63,7 @@ def split(self, desired_num_offsets_per_split, min_num_offsets_per_split=1): remaining = self.stop - current_split_stop # Avoiding a small split at the end. - if (remaining < desired_num_offsets_per_split / 4 or + if (remaining < old_div(desired_num_offsets_per_split, 4) or remaining < min_num_offsets_per_split): current_split_stop = self.stop @@ -87,9 +91,9 @@ def __init__(self, start, end): raise ValueError('Start offset must not be \'None\'') if end is None: raise ValueError('End offset must not be \'None\'') - assert isinstance(start, (int, long)) + assert isinstance(start, int) if end != self.OFFSET_INFINITY: - assert isinstance(end, (int, long)) + assert isinstance(end, int) assert start <= end @@ -163,7 +167,7 @@ def set_current_position(self, record_start): self._last_record_start = record_start def try_split(self, split_offset): - assert isinstance(split_offset, (int, long)) + assert isinstance(split_offset, int) with self._lock: if self._stop_offset == OffsetRangeTracker.OFFSET_INFINITY: logging.debug('refusing to split %r at %d: stop position unspecified', @@ -188,8 +192,8 @@ def try_split(self, split_offset): logging.debug('Agreeing to split %r at %d', self, split_offset) - split_fraction = (float(split_offset - self._start_offset) / ( - self._stop_offset - self._start_offset)) + split_fraction = (old_div(float(split_offset - self._start_offset), ( + self._stop_offset - self._start_offset))) self._stop_offset = split_offset return self._stop_offset, split_fraction @@ -425,7 +429,7 @@ def position_to_fraction(cls, key, start=None, end=None): istart = cls._string_to_int(start, prec) ikey = cls._string_to_int(key, prec) iend = cls._string_to_int(end, prec) if end else 1 << (prec * 8) - return float(ikey - istart) / (iend - istart) + return old_div(float(ikey - istart), (iend - istart)) @staticmethod def _string_to_int(s, prec): diff --git a/sdks/python/apache_beam/io/range_trackers_test.py b/sdks/python/apache_beam/io/range_trackers_test.py index 762d6547891e..199e76f6fe0e 100644 --- a/sdks/python/apache_beam/io/range_trackers_test.py +++ b/sdks/python/apache_beam/io/range_trackers_test.py @@ -16,12 +16,15 @@ # """Unit tests for the range_trackers module.""" +from __future__ import division import copy import logging import math import unittest +from past.utils import old_div + from apache_beam.io import range_trackers from apache_beam.io.range_trackers import OffsetRange @@ -136,11 +139,11 @@ def test_get_position_for_fraction_dense(self): tracker = range_trackers.OffsetRangeTracker(3, 6) # Position must be an integer type. - self.assertTrue(isinstance(tracker.position_at_fraction(0.0), (int, long))) + self.assertTrue(isinstance(tracker.position_at_fraction(0.0), int)) # [3, 3) represents 0.0 of [3, 6) self.assertEqual(3, tracker.position_at_fraction(0.0)) # [3, 4) represents up to 1/3 of [3, 6) - self.assertEqual(4, tracker.position_at_fraction(1.0 / 6)) + self.assertEqual(4, tracker.position_at_fraction(old_div(1.0, 6))) self.assertEqual(4, tracker.position_at_fraction(0.333)) # [3, 5) represents up to 2/3 of [3, 6) self.assertEqual(5, tracker.position_at_fraction(0.334)) @@ -154,9 +157,9 @@ def test_get_fraction_consumed_dense(self): self.assertTrue(tracker.try_claim(3)) self.assertEqual(0.0, tracker.fraction_consumed()) self.assertTrue(tracker.try_claim(4)) - self.assertEqual(1.0 / 3, tracker.fraction_consumed()) + self.assertEqual(old_div(1.0, 3), tracker.fraction_consumed()) self.assertTrue(tracker.try_claim(5)) - self.assertEqual(2.0 / 3, tracker.fraction_consumed()) + self.assertEqual(old_div(2.0, 3), tracker.fraction_consumed()) tracker.set_current_position(6) self.assertEqual(1.0, tracker.fraction_consumed()) tracker.set_current_position(7) @@ -197,7 +200,7 @@ def test_try_split_points(self): tracker = range_trackers.OffsetRangeTracker(100, 400) def dummy_callback(stop_position): - return int(stop_position / 5) + return int(old_div(stop_position, 5)) tracker.set_split_points_unclaimed_callback(dummy_callback) @@ -233,7 +236,7 @@ def fraction_to_position(fraction, start, end): @staticmethod def position_to_fraction(pos, start, end): - return float(pos - start) / (end - start) + return old_div(float(pos - start), (end - start)) def test_try_claim(self): tracker = self.DoubleRangeTracker(10, 20) @@ -357,15 +360,17 @@ def _check(self, fraction=None, key=None, start=None, end=None, delta=0): self.assertEqual(computed_key, key, str(locals())) def test_key_to_fraction_no_endpoints(self): - self._check(key='\x07', fraction=7/256.) - self._check(key='\xFF', fraction=255/256.) - self._check(key='\x01\x02\x03', fraction=(2**16 + 2**9 + 3) / (2.0**24)) + self._check(key='\x07', fraction=old_div(7, 256.)) + self._check(key='\xFF', fraction=old_div(255, 256.)) + self._check(key='\x01\x02\x03', + fraction=old_div((2**16 + 2**9 + 3), (2.0**24))) def test_key_to_fraction(self): - self._check(key='\x87', start='\x80', fraction=7/128.) - self._check(key='\x07', end='\x10', fraction=7/16.) - self._check(key='\x47', start='\x40', end='\x80', fraction=7/64.) - self._check(key='\x47\x80', start='\x40', end='\x80', fraction=15/128.) + self._check(key='\x87', start='\x80', fraction=old_div(7, 128.)) + self._check(key='\x07', end='\x10', fraction=old_div(7, 16.)) + self._check(key='\x47', start='\x40', end='\x80', fraction=old_div(7, 64.)) + self._check(key='\x47\x80', start='\x40', end='\x80', + fraction=old_div(15, 128.)) def test_key_to_fraction_common_prefix(self): self._check( @@ -388,7 +393,7 @@ def test_tiny(self): self._check(fraction=.5**20, start='xy_a', end='xy_c', key='xy_a\0\0\x20') self._check(fraction=.5**20, start='\xFF\xFF\x80', key='\xFF\xFF\x80\x00\x08') - self._check(fraction=.5**20 / 3, + self._check(fraction=old_div(.5**20, 3), start='xy_a', end='xy_c', key='xy_a\x00\x00\n\xaa\xaa\xaa\xaa\xaa', @@ -396,7 +401,7 @@ def test_tiny(self): self._check(fraction=.5**100, key='\0' * 12 + '\x10') def test_lots(self): - for fraction in (0, 1, .5, .75, 7./512, 1 - 7./4096): + for fraction in (0, 1, .5, .75, old_div(7., 512), 1 - old_div(7., 4096)): self._check(fraction) self._check(fraction, start='\x01') self._check(fraction, end='\xF0') @@ -406,7 +411,8 @@ def test_lots(self): self._check(fraction, start='a' * 100 + '\x80', end='a' * 100 + '\x81') self._check(fraction, start='a' * 101 + '\x80', end='a' * 101 + '\x81') self._check(fraction, start='a' * 102 + '\x80', end='a' * 102 + '\x81') - for fraction in (.3, 1/3., 1/math.e, .001, 1e-30, .99, .999999): + for fraction in ( + .3, old_div(1, 3.), old_div(1, math.e), .001, 1e-30, .99, .999999): self._check(fraction, delta=1e-14) self._check(fraction, start='\x01', delta=1e-14) self._check(fraction, end='\xF0', delta=1e-14) @@ -419,19 +425,19 @@ def test_lots(self): def test_good_prec(self): # There should be about 7 characters (~53 bits) of precision # (beyond the common prefix of start and end). - self._check(1 / math.e, start='abc_abc', end='abc_xyz', + self._check(old_div(1, math.e), start='abc_abc', end='abc_xyz', key='abc_i\xe0\xf4\x84\x86\x99\x96', delta=1e-15) # This remains true even if the start and end keys are given to # high precision. - self._check(1 / math.e, + self._check(old_div(1, math.e), start='abcd_abc\0\0\0\0\0_______________abc', end='abcd_xyz\0\0\0\0\0\0_______________abc', key='abcd_i\xe0\xf4\x84\x86\x99\x96', delta=1e-15) # For very small fractions, however, higher precision is used to # accurately represent small increments in the keyspace. - self._check(1e-20 / math.e, start='abcd_abc', end='abcd_xyz', + self._check(old_div(1e-20, math.e), start='abcd_abc', end='abcd_xyz', key='abcd_abc\x00\x00\x00\x00\x00\x01\x91#\x172N\xbb', delta=1e-35) diff --git a/sdks/python/apache_beam/io/source_test_utils.py b/sdks/python/apache_beam/io/source_test_utils.py index bea970850838..a96977603f9b 100644 --- a/sdks/python/apache_beam/io/source_test_utils.py +++ b/sdks/python/apache_beam/io/source_test_utils.py @@ -43,13 +43,17 @@ * apache_beam.io.source_test_utils_test.py * apache_beam.io.avroio_test.py """ +from __future__ import division -from collections import namedtuple import logging import threading import weakref - +from builtins import next, object, range +from collections import namedtuple from multiprocessing.pool import ThreadPool + +from past.utils import old_div + from apache_beam.io import iobase __all__ = ['read_from_source', 'assert_sources_equal_reference_source', @@ -459,7 +463,7 @@ def assert_split_at_fraction_binary( # This prevents infinite recursion. return - middle_fraction = (left_fraction + right_fraction) / 2 + middle_fraction = old_div((left_fraction + right_fraction), 2) if left_result is None: left_result = _assert_split_at_fraction_behavior( diff --git a/sdks/python/apache_beam/io/source_test_utils_test.py b/sdks/python/apache_beam/io/source_test_utils_test.py index 00522c9fd3f5..3a271b382c93 100644 --- a/sdks/python/apache_beam/io/source_test_utils_test.py +++ b/sdks/python/apache_beam/io/source_test_utils_test.py @@ -18,9 +18,10 @@ import logging import tempfile import unittest +from builtins import range -from apache_beam.io.filebasedsource_test import LineSource import apache_beam.io.source_test_utils as source_test_utils +from apache_beam.io.filebasedsource_test import LineSource class SourceTestUtilsTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/sources_test.py b/sdks/python/apache_beam/io/sources_test.py index 10d401b8d2a2..76df095c0c34 100644 --- a/sdks/python/apache_beam/io/sources_test.py +++ b/sdks/python/apache_beam/io/sources_test.py @@ -23,13 +23,10 @@ import unittest import apache_beam as beam - from apache_beam import coders -from apache_beam.io import iobase -from apache_beam.io import range_trackers +from apache_beam.io import iobase, range_trackers from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.testing.util import assert_that, equal_to class LineSource(iobase.BoundedSource): diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 9708df7c753c..1f58fa380282 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -19,20 +19,24 @@ from __future__ import absolute_import -from functools import partial + import logging +import sys +from builtins import object, range +from functools import partial from apache_beam.coders import coders -from apache_beam.io import filebasedsource -from apache_beam.io import filebasedsink -from apache_beam.io import iobase +from apache_beam.io import filebasedsink, filebasedsource, iobase from apache_beam.io.filebasedsource import ReadAllFiles from apache_beam.io.filesystem import CompressionTypes -from apache_beam.io.iobase import Read -from apache_beam.io.iobase import Write +from apache_beam.io.iobase import Read, Write from apache_beam.transforms import PTransform from apache_beam.transforms.display import DisplayDataItem +if sys.version_info[0] >= 3: + basestring = str + + __all__ = ['ReadFromText', 'ReadAllFromText', 'WriteToText'] @@ -62,7 +66,7 @@ def data(self): @data.setter def data(self, value): - assert isinstance(value, bytes) + assert isinstance(value, basestring) self._data = value @property @@ -71,7 +75,7 @@ def position(self): @position.setter def position(self, value): - assert isinstance(value, (int, long)) + assert isinstance(value, int) if value > len(self._data): raise ValueError('Cannot set position to %d since it\'s larger than ' 'size of data %d.', value, len(self._data)) diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index b29ca5a97137..09e3c9bf1331 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -16,6 +16,7 @@ # """Tests for textio module.""" +from __future__ import division import bz2 import glob @@ -25,28 +26,22 @@ import shutil import tempfile import unittest +from builtins import range + +from past.utils import old_div import apache_beam as beam -from apache_beam.io import iobase, ReadAllFromText import apache_beam.io.source_test_utils as source_test_utils - -# Importing following private classes for testing. -from apache_beam.io.textio import _TextSink as TextSink -from apache_beam.io.textio import _TextSource as TextSource - -from apache_beam.io.textio import ReadFromText -from apache_beam.io.textio import WriteToText - from apache_beam import coders -from apache_beam.io.filebasedsource_test import EOL -from apache_beam.io.filebasedsource_test import write_data -from apache_beam.io.filebasedsource_test import write_pattern +from apache_beam.io import ReadAllFromText, iobase +from apache_beam.io.filebasedsource_test import EOL, write_data, write_pattern from apache_beam.io.filesystem import CompressionTypes - +from apache_beam.io.textio import _TextSink as TextSink +from apache_beam.io.textio import _TextSource as TextSource +# Importing following private classes for testing. +from apache_beam.io.textio import ReadFromText, WriteToText from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to - +from apache_beam.testing.util import assert_that, equal_to from apache_beam.transforms.core import Create @@ -275,7 +270,7 @@ def test_progress(self): split_points_report.append(range_tracker.split_points()) self.assertEqual( - [float(i) / 10 for i in range(0, 10)], fraction_consumed_report) + [old_div(float(i), 10) for i in range(0, 10)], fraction_consumed_report) expected_split_points_report = [ ((i - 1), iobase.RangeTracker.SPLIT_POINTS_UNKNOWN) for i in range(1, 10)] diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index d7eb932cd655..01838a11afc1 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -20,15 +20,21 @@ import logging import struct +import sys +from builtins import object + +import crcmod from apache_beam import coders -from apache_beam.io import filebasedsource -from apache_beam.io import filebasedsink +from apache_beam.io import filebasedsink, filebasedsource from apache_beam.io.filesystem import CompressionTypes -from apache_beam.io.iobase import Read -from apache_beam.io.iobase import Write +from apache_beam.io.iobase import Read, Write from apache_beam.transforms import PTransform -import crcmod + +reload(sys) +if sys.version_info[0] < 3: + sys.setdefaultencoding('latin-1') + __all__ = ['ReadFromTFRecord', 'WriteToTFRecord'] @@ -90,12 +96,15 @@ def write_record(cls, file_handle, value): file_handle: The file to write to. value: A string content of the record. """ - encoded_length = struct.pack('= 3: + basestring = str __all__ = ['Pipeline'] @@ -132,11 +137,12 @@ def __init__(self, runner=None, options=None, argv=None): logging.info(('Missing pipeline option (runner). Executing pipeline ' 'using the default runner: %s.'), runner) - if isinstance(runner, str): + if isinstance(runner, basestring): runner = create_runner(runner) elif not isinstance(runner, PipelineRunner): - raise TypeError('Runner must be a PipelineRunner object or the ' - 'name of a registered runner.') + raise TypeError('Runner {0} of type {1} must be a PipelineRunner' + ' object or the name of a registered runner.' + .format(runner, type(runner))) # Validate pipeline options errors = PipelineOptionsValidator(self._options, runner).validate() @@ -549,7 +555,7 @@ def from_runner_api(proto, runner, options): context.transforms.get_by_id(root_transform_id)] # TODO(robertwb): These are only needed to continue construction. Omit? p.applied_labels = set([ - t.unique_name for t in proto.components.transforms.values()]) + t.unique_name for t in list(proto.components.transforms.values())]) for id in proto.components.pcollections: pcollection = context.pcollections.get_by_id(id) pcollection.pipeline = p @@ -679,7 +685,7 @@ def is_composite(self): is not a producer is one that returns its inputs instead.) """ return bool(self.parts) or all( - pval.producer is not self for pval in self.outputs.values()) + pval.producer is not self for pval in list(self.outputs.values())) def visit(self, visitor, pipeline, visited): """Visits all nodes reachable from the current node.""" @@ -719,7 +725,7 @@ def visit(self, visitor, pipeline, visited): # output of such a transform is the containing DoOutputsTuple, not the # PCollection inside it. Without the code below a tagged PCollection will # not be marked as visited while visiting its producer. - for pval in self.outputs.values(): + for pval in list(self.outputs.values()): if isinstance(pval, pvalue.DoOutputsTuple): pvals = (v for v in pval) else: @@ -735,7 +741,7 @@ def named_inputs(self): if isinstance(input, pvalue.PCollection)} def named_outputs(self): - return {str(tag): output for tag, output in self.outputs.items() + return {str(tag): output for tag, output in list(self.outputs.items()) if isinstance(output, pvalue.PCollection)} def to_runner_api(self, context): @@ -753,9 +759,9 @@ def transform_to_runner_api(transform, context): for part in self.parts], # TODO(BEAM-115): Side inputs. inputs={tag: context.pcollections.get_id(pc) - for tag, pc in self.named_inputs().items()}, + for tag, pc in list(self.named_inputs().items())}, outputs={str(tag): context.pcollections.get_id(out) - for tag, out in self.named_outputs().items()}, + for tag, out in list(self.named_outputs().items())}, # TODO(BEAM-115): display_data display_data=None) @@ -771,13 +777,13 @@ def from_runner_api(proto, context): context.transforms.get_by_id(id) for id in proto.subtransforms] result.outputs = { None if tag == 'None' else tag: context.pcollections.get_by_id(id) - for tag, id in proto.outputs.items()} + for tag, id in list(proto.outputs.items())} # This annotation is expected by some runners. if proto.spec.urn == urns.PARDO_TRANSFORM: result.transform.output_tags = set(proto.outputs.keys()).difference( {'None'}) if not result.parts: - for tag, pc in result.outputs.items(): + for tag, pc in list(result.outputs.items()): if pc not in result.inputs: pc.producer = result pc.tag = tag @@ -785,7 +791,7 @@ def from_runner_api(proto, context): return result -class PTransformOverride(object): +class PTransformOverride(with_metaclass(abc.ABCMeta, object)): """For internal use only; no backwards-compatibility guarantees. Gives a matcher and replacements for matching PTransforms. @@ -793,7 +799,6 @@ class PTransformOverride(object): TODO: Update this to support cases where input and/our output types are different. """ - __metaclass__ = abc.ABCMeta @abc.abstractmethod def get_matcher(self): diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index b3ac100780fe..dee80730cd54 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -20,35 +20,26 @@ import logging import platform import unittest - -# TODO(BEAM-1555): Test is failing on the service, with FakeSource. -# from nose.plugins.attrib import attr +from builtins import object, range import apache_beam as beam from apache_beam.io import Read from apache_beam.metrics import Metrics -from apache_beam.pipeline import Pipeline -from apache_beam.pipeline import PTransformOverride -from apache_beam.pipeline import PipelineOptions -from apache_beam.pipeline import PipelineVisitor +from apache_beam.pipeline import (Pipeline, PipelineOptions, PipelineVisitor, + PTransformOverride) from apache_beam.pvalue import AsSingleton from apache_beam.runners import DirectRunner from apache_beam.runners.dataflow.native_io.iobase import NativeSource from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to -from apache_beam.transforms import CombineGlobally -from apache_beam.transforms import Create -from apache_beam.transforms import FlatMap -from apache_beam.transforms import Map -from apache_beam.transforms import DoFn -from apache_beam.transforms import ParDo -from apache_beam.transforms import PTransform -from apache_beam.transforms import WindowInto -from apache_beam.transforms.window import SlidingWindows -from apache_beam.transforms.window import TimestampedValue +from apache_beam.testing.util import assert_that, equal_to +from apache_beam.transforms import (CombineGlobally, Create, DoFn, FlatMap, + Map, ParDo, PTransform, WindowInto) +from apache_beam.transforms.window import SlidingWindows, TimestampedValue from apache_beam.utils.timestamp import MIN_TIMESTAMP +# TODO(BEAM-1555): Test is failing on the service, with FakeSource. +# from nose.plugins.attrib import attr + class FakeSource(NativeSource): """Fake source returning a fixed list of values.""" @@ -457,11 +448,11 @@ def test_dir(self): options = Breakfast() self.assertEquals( set(['from_dictionary', 'get_all_options', 'slices', 'style', - 'view_as', 'display_data']), + 'view_as', 'display_data', 'next']), set([attr for attr in dir(options) if not attr.startswith('_')])) self.assertEquals( set(['from_dictionary', 'get_all_options', 'style', 'view_as', - 'display_data']), + 'display_data', 'next']), set([attr for attr in dir(options.view_as(Eggs)) if not attr.startswith('_')])) diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 34a483e7bb9c..73a8e76baac5 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -27,9 +27,11 @@ from __future__ import absolute_import import itertools +from builtins import hex, object -from apache_beam import typehints +from past.builtins import basestring +from apache_beam import typehints __all__ = [ 'PCollection', diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py index 84535692806b..7872453dcfa8 100644 --- a/sdks/python/apache_beam/runners/common.py +++ b/sdks/python/apache_beam/runners/common.py @@ -24,14 +24,16 @@ import sys import traceback +from builtins import object, zip + +from past.builtins import basestring from apache_beam.internal import util from apache_beam.metrics.execution import ScopedMetricsContainer from apache_beam.pvalue import TaggedOutput from apache_beam.transforms import core -from apache_beam.transforms.window import TimestampedValue -from apache_beam.transforms.window import WindowFn -from apache_beam.transforms.window import GlobalWindow +from apache_beam.transforms.window import (GlobalWindow, TimestampedValue, + WindowFn) from apache_beam.utils.windowed_value import WindowedValue @@ -248,14 +250,14 @@ def __init__(self, placeholder): elif d == core.DoFn.SideInputParam: # If no more args are present then the value must be passed via kwarg try: - args_with_placeholders.append(remaining_args_iter.next()) + args_with_placeholders.append(next(remaining_args_iter)) except StopIteration: if a not in input_kwargs: raise ValueError("Value for sideinput %s not provided" % a) else: # If no more args are present then the value must be passed via kwarg try: - args_with_placeholders.append(remaining_args_iter.next()) + args_with_placeholders.append(next(remaining_args_iter)) except StopIteration: pass args_with_placeholders.extend(list(remaining_args_iter)) diff --git a/sdks/python/apache_beam/runners/common_test.py b/sdks/python/apache_beam/runners/common_test.py index 62a6955f6ce4..e0f628c71ee1 100644 --- a/sdks/python/apache_beam/runners/common_test.py +++ b/sdks/python/apache_beam/runners/common_test.py @@ -17,8 +17,8 @@ import unittest -from apache_beam.transforms.core import DoFn from apache_beam.runners.common import DoFnSignature +from apache_beam.transforms.core import DoFn class DoFnSignatureTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_metrics.py b/sdks/python/apache_beam/runners/dataflow/dataflow_metrics.py index 24916fd49278..f7a931ad97af 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_metrics.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_metrics.py @@ -21,13 +21,11 @@ service. """ -from collections import defaultdict import numbers +from collections import defaultdict -from apache_beam.metrics.cells import DistributionData -from apache_beam.metrics.cells import DistributionResult -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.execution import MetricResult +from apache_beam.metrics.cells import DistributionData, DistributionResult +from apache_beam.metrics.execution import MetricKey, MetricResult from apache_beam.metrics.metric import MetricResults from apache_beam.metrics.metricbase import MetricName @@ -145,7 +143,7 @@ def _populate_metric_results(self, response): # Now we create the MetricResult elements. result = [] - for metric_key, metric in metrics_by_name.iteritems(): + for metric_key, metric in metrics_by_name.items(): attempted = self._get_metric_value(metric['tentative']) committed = self._get_metric_value(metric['committed']) if attempted is None or committed is None: diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_metrics_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_metrics_test.py index dd3cbe1156a4..5c00fce33226 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_metrics_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_metrics_test.py @@ -20,13 +20,12 @@ """ import types import unittest +from builtins import object import mock -from apache_beam.metrics.cells import DistributionData -from apache_beam.metrics.cells import DistributionResult -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.execution import MetricResult +from apache_beam.metrics.cells import DistributionData, DistributionResult +from apache_beam.metrics.execution import MetricKey, MetricResult from apache_beam.metrics.metricbase import MetricName from apache_beam.runners.dataflow import dataflow_metrics @@ -34,7 +33,7 @@ class DictToObject(object): """Translate from a dict(list()) structure to an object structure""" def __init__(self, data): - for name, value in data.iteritems(): + for name, value in data.items(): setattr(self, name, self._wrap(value)) def _wrap(self, value): diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 813759e12531..b12b57f3fd91 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -21,37 +21,41 @@ to the Dataflow Service for remote execution by a worker. """ -from collections import defaultdict import logging import threading import time import traceback -import urllib +import urllib.error +import urllib.parse +import urllib.request +from builtins import hex +from collections import defaultdict + +from future import standard_library import apache_beam as beam -from apache_beam import error -from apache_beam import coders -from apache_beam import pvalue +from apache_beam import coders, error, pvalue from apache_beam.internal import pickler from apache_beam.internal.gcp import json_value +from apache_beam.options.pipeline_options import (SetupOptions, + StandardOptions, TestOptions) from apache_beam.pvalue import AsSideInput from apache_beam.runners.dataflow.dataflow_metrics import DataflowMetrics from apache_beam.runners.dataflow.internal import names -from apache_beam.runners.dataflow.internal.clients import dataflow as dataflow_api -from apache_beam.runners.dataflow.internal.names import PropertyNames -from apache_beam.runners.dataflow.internal.names import TransformNames -from apache_beam.runners.dataflow.ptransform_overrides import CreatePTransformOverride -from apache_beam.runners.runner import PValueCache -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineRunner -from apache_beam.runners.runner import PipelineState +from apache_beam.runners.dataflow.internal.clients import \ + dataflow as dataflow_api +from apache_beam.runners.dataflow.internal.names import (PropertyNames, + TransformNames) +from apache_beam.runners.dataflow.ptransform_overrides import \ + CreatePTransformOverride +from apache_beam.runners.runner import (PipelineResult, PipelineRunner, + PipelineState, PValueCache) from apache_beam.transforms.display import DisplayData from apache_beam.typehints import typehints -from apache_beam.options.pipeline_options import SetupOptions -from apache_beam.options.pipeline_options import StandardOptions -from apache_beam.options.pipeline_options import TestOptions from apache_beam.utils.plugin import BeamPlugin +standard_library.install_aliases() + __all__ = ['DataflowRunner'] @@ -832,7 +836,8 @@ def serialize_windowing_strategy(cls, windowing): return cls.byte_array_to_json_string( beam_runner_api_pb2.MessageWithComponents( components=context.to_runner_api(), - windowing_strategy=windowing_proto).SerializeToString()) + windowing_strategy=windowing_proto).SerializeToString() + ) @classmethod def deserialize_windowing_strategy(cls, serialized_data): @@ -850,12 +855,12 @@ def deserialize_windowing_strategy(cls, serialized_data): @staticmethod def byte_array_to_json_string(raw_bytes): """Implements org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString.""" - return urllib.quote(raw_bytes) + return urllib.parse.quote(raw_bytes.decode("latin-1")) @staticmethod def json_string_to_byte_array(encoded_string): """Implements org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray.""" - return urllib.unquote(encoded_string) + return urllib.parse.unquote(encoded_string).encode("latin-1") class DataflowPipelineResult(PipelineResult): diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 8e708e6fde29..46433d965a7f 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -19,28 +19,26 @@ import json import unittest +from builtins import object, range from datetime import datetime import mock import apache_beam as beam import apache_beam.transforms as ptransform - from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.pipeline import Pipeline, AppliedPTransform +from apache_beam.pipeline import AppliedPTransform, Pipeline from apache_beam.pvalue import PCollection -from apache_beam.runners import create_runner -from apache_beam.runners import DataflowRunner -from apache_beam.runners import TestDataflowRunner -from apache_beam.runners.dataflow.dataflow_runner import DataflowPipelineResult -from apache_beam.runners.dataflow.dataflow_runner import DataflowRuntimeException -from apache_beam.runners.dataflow.internal.clients import dataflow as dataflow_api -from apache_beam.runners.runner import PipelineState +from apache_beam.runners import (DataflowRunner, PipelineState, + TestDataflowRunner, create_runner) +from apache_beam.runners.dataflow.dataflow_runner import ( + DataflowPipelineResult, DataflowRuntimeException) +from apache_beam.runners.dataflow.internal.clients import \ + dataflow as dataflow_api from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.transforms.display import DisplayDataItem -from apache_beam.transforms.core import _GroupByKeyOnly -from apache_beam.transforms.core import Windowing from apache_beam.transforms import window +from apache_beam.transforms.core import Windowing, _GroupByKeyOnly +from apache_beam.transforms.display import DisplayDataItem from apache_beam.typehints import typehints # Protect against environments where apitools library is not available. @@ -352,10 +350,9 @@ def test_serialize_windowing_strategy(self): # This just tests the basic path; more complete tests # are in window_test.py. strategy = Windowing(window.FixedWindows(10)) - self.assertEqual( - strategy, - DataflowRunner.deserialize_windowing_strategy( - DataflowRunner.serialize_windowing_strategy(strategy))) + serialized = DataflowRunner.serialize_windowing_strategy(strategy) + deserialized = DataflowRunner.deserialize_windowing_strategy(serialized) + self.assertEqual(strategy, deserialized) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index a1f9301b19ba..84c48d77aa39 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -25,28 +25,37 @@ import logging import os import re +import sys import time -from StringIO import StringIO +from builtins import object from datetime import datetime -from apitools.base.py import encoding -from apitools.base.py import exceptions +from apitools.base.py import encoding, exceptions +from future import standard_library from apache_beam.internal.gcp.auth import get_service_credentials from apache_beam.internal.gcp.json_value import to_json_value from apache_beam.io.filesystems import FileSystems from apache_beam.io.gcp.internal.clients import storage +from apache_beam.options.pipeline_options import (DebugOptions, + GoogleCloudOptions, + StandardOptions, + WorkerOptions) from apache_beam.runners.dataflow.internal import dependency from apache_beam.runners.dataflow.internal.clients import dataflow -from apache_beam.runners.dataflow.internal.dependency import get_sdk_name_and_version +from apache_beam.runners.dataflow.internal.dependency import \ + get_sdk_name_and_version from apache_beam.runners.dataflow.internal.names import PropertyNames from apache_beam.transforms import cy_combiners from apache_beam.transforms.display import DisplayData from apache_beam.utils import retry -from apache_beam.options.pipeline_options import DebugOptions -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import StandardOptions -from apache_beam.options.pipeline_options import WorkerOptions + +standard_library.install_aliases() + +if sys.version_info[0] >= 3: + from io import StringIO +else: + from StringIO import StringIO # Environment version information. It is passed to the service during a @@ -241,7 +250,7 @@ def __init__(self, packages, options, environment_version): dataflow.Environment.SdkPipelineOptionsValue()) options_dict = {k: v - for k, v in sdk_pipeline_options.iteritems() + for k, v in sdk_pipeline_options.items() if v is not None} self.proto.sdkPipelineOptions.additionalProperties.append( dataflow.Environment.SdkPipelineOptionsValue.AdditionalProperty( @@ -276,7 +285,7 @@ def encode_shortstrings(input_buffer, errors='strict'): def decode_shortstrings(input_buffer, errors='strict'): """Decoder (to Unicode) that suppresses long base64 strings.""" shortened, length = encode_shortstrings(input_buffer, errors) - return unicode(shortened), length + return str(shortened), length def shortstrings_registerer(encoding_name): if encoding_name == 'shortstrings': @@ -290,8 +299,10 @@ def shortstrings_registerer(encoding_name): # Use json "dump string" method to get readable formatting; # further modify it to not output too-long strings, aimed at the # 10,000+ character hex-encoded "serialized_fn" values. + encoded_proto = encoding.MessageToJson(self.proto) return json.dumps( - json.loads(encoding.MessageToJson(self.proto), encoding='shortstrings'), + json.loads(encoded_proto.encode("utf-8").decode("utf-8"), + encoding='shortstrings'), indent=2, sort_keys=True) @staticmethod @@ -462,7 +473,12 @@ def create_job_description(self, job): job.proto.environment = Environment( packages=resources, options=job.options, environment_version=self.environment_version).proto - logging.debug('JOB: %s', job) + if sys.version_info[0] >= 3: + logging.debug('JOB: %s', job) + else: + # Dumping to JSON after 2/3 can cause problems, skip for now. + logging.debug("JOB pkgs %s opts %s env version %s", + resources, job.options, self.environment_version) @retry.with_exponential_backoff(num_retries=3, initial_delay_secs=3) def get_job_metrics(self, job_id): diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py index 407ffcf2ad72..e7cced7b4f3f 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py @@ -21,7 +21,6 @@ from apache_beam.metrics.cells import DistributionData from apache_beam.options.pipeline_options import PipelineOptions - from apache_beam.runners.dataflow.internal.clients import dataflow # Protect against environments where apitools library is not available. diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py index 61d02730ab54..977a4c03c038 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py @@ -19,7 +19,8 @@ # NOTE: This file is autogenerated and should not be edited by hand. from apitools.base.py import base_api -from apache_beam.runners.dataflow.internal.clients.dataflow import dataflow_v1b3_messages as messages +from apache_beam.runners.dataflow.internal.clients.dataflow import \ + dataflow_v1b3_messages as messages class DataflowV1b3(base_api.BaseApiClient): diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py index e81329fca80f..6f8e3e755ca3 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py @@ -23,9 +23,7 @@ # NOTE: This file is autogenerated and should not be edited by hand. from apitools.base.protorpclite import messages as _messages -from apitools.base.py import encoding -from apitools.base.py import extra_types - +from apitools.base.py import encoding, extra_types package = 'dataflow' @@ -3069,7 +3067,6 @@ class ResourceUtilizationReportResponse(_messages.Message): """ - class RuntimeEnvironment(_messages.Message): """The environment values to set at runtime. diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers.py index 4dda47ad9e05..35c6ed242984 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers.py @@ -17,7 +17,6 @@ from hamcrest.core.base_matcher import BaseMatcher - IGNORED = object() @@ -50,7 +49,7 @@ def _matches(self, item): if self.origin != IGNORED and item.origin != self.origin: return False if self.context != IGNORED: - for key, name in self.context.iteritems(): + for key, name in self.context.items(): if key not in item.context: return False if name != IGNORED and item.context[key] != name: diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers_test.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers_test.py index 3163c9b25125..22b13f511efa 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/message_matchers_test.py @@ -15,11 +15,13 @@ # limitations under the License. # import unittest + import hamcrest as hc -import apache_beam.runners.dataflow.internal.clients.dataflow as dataflow +import apache_beam.runners.dataflow.internal.clients.dataflow as dataflow from apache_beam.internal.gcp.json_value import to_json_value -from apache_beam.runners.dataflow.internal.clients.dataflow import message_matchers +from apache_beam.runners.dataflow.internal.clients.dataflow import \ + message_matchers # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py index f5dd2bd35732..d5e61241439b 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py @@ -64,10 +64,10 @@ from apache_beam import version as beam_version from apache_beam.internal import pickler from apache_beam.io.filesystems import FileSystems +from apache_beam.options.pipeline_options import (GoogleCloudOptions, + SetupOptions) from apache_beam.runners.dataflow.internal import names from apache_beam.utils import processes -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import SetupOptions # All constants are for internal use only; no backwards-compatibility # guarantees. diff --git a/sdks/python/apache_beam/runners/dataflow/internal/dependency_test.py b/sdks/python/apache_beam/runners/dataflow/internal/dependency_test.py index e555b69f15ad..83fecb189109 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/dependency_test.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/dependency_test.py @@ -24,12 +24,10 @@ import unittest from apache_beam.io.filesystems import FileSystems -from apache_beam.runners.dataflow.internal import dependency -from apache_beam.runners.dataflow.internal import names -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import SetupOptions - +from apache_beam.options.pipeline_options import (GoogleCloudOptions, + PipelineOptions, + SetupOptions) +from apache_beam.runners.dataflow.internal import dependency, names # Protect against environments where GCS library is not available. # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index be6722427b6c..fedc211ca2ea 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -19,6 +19,8 @@ # Standard file names used for staging files. +from builtins import object + PICKLED_MAIN_SESSION_FILE = 'pickled_main_session' DATAFLOW_SDK_TARBALL_FILE = 'dataflow_python_sdk.tar' diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py index 2f2316f6f1d0..5ee3c221056b 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py @@ -21,6 +21,7 @@ """ import logging +from builtins import object from apache_beam import pvalue from apache_beam.io import iobase @@ -31,7 +32,7 @@ def _dict_printable_fields(dict_object, skip_fields): """Returns a list of strings for the interesting fields of a dict.""" return ['%s=%r' % (name, value) - for name, value in dict_object.iteritems() + for name, value in dict_object.items() # want to output value 0 but not None nor [] if (value or value == 0) and name not in skip_fields] diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py index 3d8c24f5651c..0fa47f9793b4 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py @@ -20,21 +20,11 @@ import unittest -from apache_beam import Create -from apache_beam import error -from apache_beam import pvalue +from apache_beam import Create, error, pvalue from apache_beam.runners.dataflow.native_io.iobase import ( - _dict_printable_fields, - _NativeWrite, - ConcatPosition, - DynamicSplitRequest, - DynamicSplitResultWithPosition, - NativeSink, - NativeSinkWriter, - NativeSource, - ReaderPosition, - ReaderProgress -) + ConcatPosition, DynamicSplitRequest, DynamicSplitResultWithPosition, + NativeSink, NativeSinkWriter, NativeSource, ReaderPosition, + ReaderProgress, _dict_printable_fields, _NativeWrite) from apache_beam.testing.test_pipeline import TestPipeline diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/streaming_create.py b/sdks/python/apache_beam/runners/dataflow/native_io/streaming_create.py index 8c6c8d6d5299..533a160616c7 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/streaming_create.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/streaming_create.py @@ -17,11 +17,9 @@ """Create transform for streaming.""" -from apache_beam import pvalue -from apache_beam import DoFn -from apache_beam import ParDo -from apache_beam import PTransform -from apache_beam import Windowing +from builtins import map + +from apache_beam import DoFn, ParDo, PTransform, Windowing, pvalue from apache_beam.transforms.window import GlobalWindows @@ -34,7 +32,7 @@ class StreamingCreate(PTransform): def __init__(self, values, coder): self.coder = coder - self.encoded_values = map(coder.encode, values) + self.encoded_values = list(map(coder.encode, values)) class DecodeAndEmitDoFn(DoFn): """A DoFn which stores encoded versions of elements. diff --git a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py index 7927219ab5ea..88afe8a2ee62 100644 --- a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py @@ -24,8 +24,8 @@ import unittest import apache_beam as beam -from apache_beam.pipeline import Pipeline from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.pipeline import Pipeline from apache_beam.runners.dataflow.dataflow_runner import DataflowRunner # Protect against environments where apitools library is not available. diff --git a/sdks/python/apache_beam/runners/dataflow/test_dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/test_dataflow_runner.py index 96e6a66caab4..12ad72eba385 100644 --- a/sdks/python/apache_beam/runners/dataflow/test_dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/test_dataflow_runner.py @@ -16,12 +16,13 @@ # """Wrapper of Beam runners that's built for running and verifying e2e tests.""" +from __future__ import print_function from apache_beam.internal import pickler -from apache_beam.options.pipeline_options import TestOptions, GoogleCloudOptions +from apache_beam.options.pipeline_options import (GoogleCloudOptions, + TestOptions) from apache_beam.runners.dataflow.dataflow_runner import DataflowRunner - __all__ = ['TestDataflowRunner'] diff --git a/sdks/python/apache_beam/runners/direct/bundle_factory.py b/sdks/python/apache_beam/runners/direct/bundle_factory.py index 0182b4c9e9ca..5a35dc675b9a 100644 --- a/sdks/python/apache_beam/runners/direct/bundle_factory.py +++ b/sdks/python/apache_beam/runners/direct/bundle_factory.py @@ -19,6 +19,8 @@ from __future__ import absolute_import +from builtins import object + from apache_beam import pvalue from apache_beam.utils.windowed_value import WindowedValue diff --git a/sdks/python/apache_beam/runners/direct/clock.py b/sdks/python/apache_beam/runners/direct/clock.py index 84d52f79948b..e019af27fb1b 100644 --- a/sdks/python/apache_beam/runners/direct/clock.py +++ b/sdks/python/apache_beam/runners/direct/clock.py @@ -20,6 +20,7 @@ from __future__ import absolute_import import time +from builtins import object class Clock(object): diff --git a/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor_test.py b/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor_test.py index 97d1ee8bfb6b..0263904a7341 100644 --- a/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor_test.py +++ b/sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor_test.py @@ -21,18 +21,14 @@ import unittest from apache_beam import pvalue -from apache_beam.io import iobase -from apache_beam.io import Read +from apache_beam.io import Read, iobase from apache_beam.pipeline import Pipeline from apache_beam.pvalue import AsList from apache_beam.runners.direct import DirectRunner -from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import ConsumerTrackingPipelineVisitor -from apache_beam.transforms import CoGroupByKey -from apache_beam.transforms import Create -from apache_beam.transforms import DoFn -from apache_beam.transforms import FlatMap -from apache_beam.transforms import Flatten -from apache_beam.transforms import ParDo +from apache_beam.runners.direct.consumer_tracking_pipeline_visitor import \ + ConsumerTrackingPipelineVisitor +from apache_beam.transforms import (CoGroupByKey, Create, DoFn, FlatMap, + Flatten, ParDo) # Disable frequent lint warning due to pipe operator for chaining transforms. # pylint: disable=expression-not-assigned diff --git a/sdks/python/apache_beam/runners/direct/direct_metrics.py b/sdks/python/apache_beam/runners/direct/direct_metrics.py index 9d234876035a..5500b083176f 100644 --- a/sdks/python/apache_beam/runners/direct/direct_metrics.py +++ b/sdks/python/apache_beam/runners/direct/direct_metrics.py @@ -20,13 +20,12 @@ responding to queries of current metrics, but also of keeping the common state consistent. """ -from collections import defaultdict import threading +from builtins import object +from collections import defaultdict -from apache_beam.metrics.cells import CounterAggregator -from apache_beam.metrics.cells import DistributionAggregator -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.execution import MetricResult +from apache_beam.metrics.cells import CounterAggregator, DistributionAggregator +from apache_beam.metrics.execution import MetricKey, MetricResult from apache_beam.metrics.metric import MetricResults @@ -38,10 +37,10 @@ def __init__(self): lambda: DirectMetric(DistributionAggregator())) def _apply_operation(self, bundle, updates, op): - for k, v in updates.counters.items(): + for k, v in list(updates.counters.items()): op(self._counters[k], bundle, v) - for k, v in updates.distributions.items(): + for k, v in list(updates.distributions.items()): op(self._distributions[k], bundle, v) def commit_logical(self, bundle, updates): @@ -60,12 +59,12 @@ def query(self, filter=None): counters = [MetricResult(MetricKey(k.step, k.metric), v.extract_committed(), v.extract_latest_attempted()) - for k, v in self._counters.items() + for k, v in list(self._counters.items()) if self.matches(filter, k)] distributions = [MetricResult(MetricKey(k.step, k.metric), v.extract_committed(), v.extract_latest_attempted()) - for k, v in self._distributions.items() + for k, v in list(self._distributions.items()) if self.matches(filter, k)] return {'counters': counters, @@ -106,7 +105,7 @@ def extract_committed(self): def extract_latest_attempted(self): res = self.finished_attempted - for _, u in self.inflight_attempted.items(): + for _, u in list(self.inflight_attempted.items()): res = self.aggregator.combine(res, u) return self.aggregator.result(res) diff --git a/sdks/python/apache_beam/runners/direct/direct_metrics_test.py b/sdks/python/apache_beam/runners/direct/direct_metrics_test.py index 256b91f3e9af..a9b3120d1dda 100644 --- a/sdks/python/apache_beam/runners/direct/direct_metrics_test.py +++ b/sdks/python/apache_beam/runners/direct/direct_metrics_test.py @@ -19,12 +19,10 @@ import hamcrest as hc +from apache_beam.metrics.cells import DistributionData, DistributionResult +from apache_beam.metrics.execution import (MetricKey, MetricResult, + MetricUpdates) from apache_beam.metrics.metricbase import MetricName -from apache_beam.metrics.execution import MetricUpdates -from apache_beam.metrics.execution import MetricResult -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.cells import DistributionData -from apache_beam.metrics.cells import DistributionResult from apache_beam.runners.direct.direct_metrics import DirectMetrics diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 2deb7dace023..22dadff3923c 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -25,25 +25,21 @@ import collections import logging +from builtins import object from google.protobuf import wrappers_pb2 import apache_beam as beam from apache_beam import typehints from apache_beam.metrics.execution import MetricsEnvironment +from apache_beam.options.pipeline_options import DirectOptions, StandardOptions +from apache_beam.options.value_provider import RuntimeValueProvider from apache_beam.pvalue import PCollection from apache_beam.runners.direct.bundle_factory import BundleFactory -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineRunner -from apache_beam.runners.runner import PipelineState -from apache_beam.runners.runner import PValueCache +from apache_beam.runners.runner import (PipelineResult, PipelineRunner, + PipelineState, PValueCache) +from apache_beam.transforms.core import _GroupAlsoByWindow, _GroupByKeyOnly from apache_beam.transforms.ptransform import PTransform -from apache_beam.transforms.core import _GroupAlsoByWindow -from apache_beam.transforms.core import _GroupByKeyOnly -from apache_beam.options.pipeline_options import DirectOptions -from apache_beam.options.pipeline_options import StandardOptions -from apache_beam.options.value_provider import RuntimeValueProvider - __all__ = ['DirectRunner'] @@ -100,6 +96,7 @@ class DirectRunner(PipelineRunner): def __init__(self): self._cache = None + super(DirectRunner, self).__init__() def apply_CombinePerKey(self, transform, pcoll): # TODO: Move imports to top. Pipeline <-> Runner dependency cause problems @@ -140,7 +137,7 @@ def apply_ReadStringsFromPubSub(self, transform, pcoll): 'apache_beam[gcp]') # Execute this as a native transform. output = PCollection(pcoll.pipeline) - output.element_type = unicode + output.element_type = str return output def apply_WriteStringsToPubSub(self, transform, pcoll): @@ -181,7 +178,7 @@ def _flush(self): self._buffer = [] output = pcoll | beam.ParDo(DirectWriteToPubSub(project, topic_name)) - output.element_type = unicode + output.element_type = str return output def run(self, pipeline): @@ -269,7 +266,7 @@ def append(self, applied_ptransform, tag, elements): def finalize(self): """Make buffered cache elements visible to the underlying PValueCache.""" assert not self._finalized - for key, value in self._cache.iteritems(): + for key, value in self._cache.items(): applied_ptransform, tag = key self._pvalue_cache.cache_output(applied_ptransform, tag, value) self._cache = None diff --git a/sdks/python/apache_beam/runners/direct/evaluation_context.py b/sdks/python/apache_beam/runners/direct/evaluation_context.py index 54c407c1c866..f207c612695e 100644 --- a/sdks/python/apache_beam/runners/direct/evaluation_context.py +++ b/sdks/python/apache_beam/runners/direct/evaluation_context.py @@ -21,12 +21,13 @@ import collections import threading +from builtins import object -from apache_beam.transforms import sideinputs from apache_beam.runners.direct.clock import Clock -from apache_beam.runners.direct.watermark_manager import WatermarkManager -from apache_beam.runners.direct.executor import TransformExecutor from apache_beam.runners.direct.direct_metrics import DirectMetrics +from apache_beam.runners.direct.executor import TransformExecutor +from apache_beam.runners.direct.watermark_manager import WatermarkManager +from apache_beam.transforms import sideinputs from apache_beam.transforms.trigger import InMemoryUnmergedState from apache_beam.utils import counters @@ -162,7 +163,7 @@ def _initialize_keyed_states(self, root_transforms, value_to_consumers): transform_keyed_states = {} for transform in root_transforms: transform_keyed_states[transform] = {} - for consumers in value_to_consumers.values(): + for consumers in list(value_to_consumers.values()): for consumer in consumers: transform_keyed_states[consumer] = {} return transform_keyed_states diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index 3e08b522d541..1571b33646fb 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -22,16 +22,21 @@ import collections import itertools import logging -import Queue +import queue import sys import threading import traceback +from builtins import object, range from weakref import WeakValueDictionary -from apache_beam.metrics.execution import MetricsContainer -from apache_beam.metrics.execution import ScopedMetricsContainer +from future import standard_library + +from apache_beam.metrics.execution import (MetricsContainer, + ScopedMetricsContainer) from apache_beam.options.pipeline_options import DirectOptions +standard_library.install_aliases() + class _ExecutorService(object): """Thread pool for executing tasks in parallel.""" @@ -77,7 +82,7 @@ def _get_task_or_none(self): # shutdown. return self.queue.get( timeout=_ExecutorService._ExecutorServiceWorker.TIMEOUT) - except Queue.Empty: + except queue.Empty: return None def run(self): @@ -96,7 +101,7 @@ def shutdown(self): self.shutdown_requested = True def __init__(self, num_workers): - self.queue = Queue.Queue() + self.queue = queue.Queue() self.workers = [_ExecutorService._ExecutorServiceWorker( self.queue, i) for i in range(num_workers)] self.shutdown_requested = False @@ -121,7 +126,7 @@ def shutdown(self): try: self.queue.get_nowait() self.queue.task_done() - except Queue.Empty: + except queue.Empty: continue # All existing threads will eventually terminate (after they complete their # last task). @@ -351,7 +356,7 @@ def attempt_call(self, metrics_container, uncommitted_bundle.get_elements_iterable()) undeclared_tag_values = result.undeclared_tag_values if undeclared_tag_values: - for tag, value in undeclared_tag_values.iteritems(): + for tag, value in undeclared_tag_values.items(): self._evaluation_context.append_to_cache( self._applied_ptransform, tag, value) @@ -398,7 +403,7 @@ def start(self, roots): self.all_nodes = frozenset( itertools.chain( roots, - *itertools.chain(self.value_to_consumers.values()))) + *itertools.chain(list(self.value_to_consumers.values())))) self.node_to_pending_bundles = {} for root_node in self.root_nodes: provider = (self.transform_evaluator_registry @@ -452,14 +457,14 @@ class _TypedUpdateQueue(object): def __init__(self, item_type): self._item_type = item_type - self._queue = Queue.Queue() + self._queue = queue.Queue() def poll(self): try: item = self._queue.get_nowait() self._queue.task_done() return item - except Queue.Empty: + except queue.Empty: return None def take(self): @@ -472,7 +477,7 @@ def take(self): item = self._queue.get(timeout=1) self._queue.task_done() return item - except Queue.Empty: + except queue.Empty: pass def offer(self, item): diff --git a/sdks/python/apache_beam/runners/direct/helper_transforms.py b/sdks/python/apache_beam/runners/direct/helper_transforms.py index 374cd4ea69db..26b0701bd02b 100644 --- a/sdks/python/apache_beam/runners/direct/helper_transforms.py +++ b/sdks/python/apache_beam/runners/direct/helper_transforms.py @@ -20,8 +20,8 @@ import apache_beam as beam from apache_beam import typehints -from apache_beam.utils.windowed_value import WindowedValue from apache_beam.internal.util import ArgumentPlaceholder +from apache_beam.utils.windowed_value import WindowedValue class LiftedCombinePerKey(beam.PTransform): diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py index cb2ace29f0ea..b35859c07d77 100644 --- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py +++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py @@ -22,37 +22,32 @@ import collections import random import time +from builtins import object -from apache_beam import coders -from apache_beam import pvalue -from apache_beam.internal import pickler import apache_beam.io as io -from apache_beam.runners.common import DoFnRunner -from apache_beam.runners.common import DoFnState -from apache_beam.runners.direct.direct_runner import _StreamingGroupByKeyOnly -from apache_beam.runners.direct.direct_runner import _StreamingGroupAlsoByWindow +from apache_beam import coders, pvalue +from apache_beam.internal import pickler +from apache_beam.options.pipeline_options import TypeOptions +from apache_beam.runners.common import DoFnRunner, DoFnState +from apache_beam.runners.dataflow.native_io.iobase import \ + _NativeWrite # pylint: disable=protected-access +from apache_beam.runners.direct.direct_runner import (_StreamingGroupAlsoByWindow, + _StreamingGroupByKeyOnly) +from apache_beam.runners.direct.util import KeyedWorkItem, TransformResult from apache_beam.runners.direct.watermark_manager import WatermarkManager -from apache_beam.runners.direct.util import KeyedWorkItem -from apache_beam.runners.direct.util import TransformResult -from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite # pylint: disable=protected-access -from apache_beam.testing.test_stream import TestStream -from apache_beam.testing.test_stream import ElementEvent -from apache_beam.testing.test_stream import WatermarkEvent -from apache_beam.testing.test_stream import ProcessingTimeEvent +from apache_beam.testing.test_stream import (ElementEvent, ProcessingTimeEvent, + TestStream, WatermarkEvent) from apache_beam.transforms import core -from apache_beam.transforms.window import GlobalWindows -from apache_beam.transforms.window import WindowedValue -from apache_beam.transforms.trigger import create_trigger_driver -from apache_beam.transforms.trigger import _CombiningValueStateTag -from apache_beam.transforms.trigger import _ListStateTag -from apache_beam.transforms.trigger import TimeDomain -from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn -from apache_beam.typehints.typecheck import TypeCheckError -from apache_beam.typehints.typecheck import TypeCheckWrapperDoFn +from apache_beam.transforms.trigger import (TimeDomain, + _CombiningValueStateTag, + _ListStateTag, + create_trigger_driver) +from apache_beam.transforms.window import GlobalWindows, WindowedValue +from apache_beam.typehints.typecheck import (OutputCheckWrapperDoFn, + TypeCheckError, + TypeCheckWrapperDoFn) from apache_beam.utils import counters -from apache_beam.utils.timestamp import Timestamp -from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.options.pipeline_options import TypeOptions +from apache_beam.utils.timestamp import MIN_TIMESTAMP, Timestamp class TransformEvaluatorRegistry(object): @@ -137,7 +132,7 @@ def should_execute_serially(self, applied_ptransform): (core._GroupByKeyOnly, _StreamingGroupByKeyOnly, _StreamingGroupAlsoByWindow, - _NativeWrite,)) + _NativeWrite)) class RootBundleProvider(object): @@ -196,7 +191,7 @@ def __init__(self, evaluation_context, applied_ptransform, def _expand_outputs(self): outputs = set() - for pval in self._applied_ptransform.outputs.values(): + for pval in list(self._applied_ptransform.outputs.values()): if isinstance(pval, pvalue.DoOutputsTuple): pvals = (v for v in pval) else: @@ -421,7 +416,7 @@ def _read_from_pubsub(self): with pubsub.subscription.AutoAck( self._subscription, return_immediately=True, max_messages=10) as results: - return [message.data for unused_ack_id, message in results.items()] + return [message.data for unused_ack_id, message in list(results.items())] def finish_bundle(self): data = self._read_from_pubsub() @@ -554,7 +549,7 @@ def process_element(self, element): def finish_bundle(self): self.runner.finish() - bundles = self._tagged_receivers.values() + bundles = list(self._tagged_receivers.values()) result_counters = self._counter_factory.get_counters() return TransformResult( self._applied_ptransform, bundles, [], result_counters, None, @@ -693,7 +688,7 @@ def process_element(self, element): def finish_bundle(self): bundles = [] bundle = None - for encoded_k, vs in self.gbk_items.iteritems(): + for encoded_k, vs in self.gbk_items.items(): if not bundle: bundle = self._evaluation_context.create_bundle( self.output_pcollection) diff --git a/sdks/python/apache_beam/runners/direct/util.py b/sdks/python/apache_beam/runners/direct/util.py index 10f7b294c130..1f745ae0d97b 100644 --- a/sdks/python/apache_beam/runners/direct/util.py +++ b/sdks/python/apache_beam/runners/direct/util.py @@ -22,6 +22,8 @@ from __future__ import absolute_import +from builtins import object + class TransformResult(object): """Result of evaluating an AppliedPTransform with a TransformEvaluator.""" diff --git a/sdks/python/apache_beam/runners/direct/watermark_manager.py b/sdks/python/apache_beam/runners/direct/watermark_manager.py index 935998d27de0..067593c6c5c2 100644 --- a/sdks/python/apache_beam/runners/direct/watermark_manager.py +++ b/sdks/python/apache_beam/runners/direct/watermark_manager.py @@ -20,13 +20,12 @@ from __future__ import absolute_import import threading +from builtins import object -from apache_beam import pipeline -from apache_beam import pvalue +from apache_beam import pipeline, pvalue from apache_beam.runners.direct.util import TimerFiring -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.utils.timestamp import TIME_GRANULARITY +from apache_beam.utils.timestamp import (MAX_TIMESTAMP, MIN_TIMESTAMP, + TIME_GRANULARITY) class WatermarkManager(object): @@ -50,12 +49,12 @@ def __init__(self, clock, root_transforms, value_to_consumers, self._transform_to_watermarks[root_transform] = _TransformWatermarks( self._clock, transform_keyed_states[root_transform], root_transform) - for consumers in value_to_consumers.values(): + for consumers in list(value_to_consumers.values()): for consumer in consumers: self._transform_to_watermarks[consumer] = _TransformWatermarks( self._clock, transform_keyed_states[consumer], consumer) - for consumers in value_to_consumers.values(): + for consumers in list(value_to_consumers.values()): for consumer in consumers: self._update_input_transform_watermarks(consumer) @@ -132,7 +131,7 @@ def _refresh_watermarks(self, applied_ptransform): assert isinstance(applied_ptransform, pipeline.AppliedPTransform) tw = self.get_watermarks(applied_ptransform) if tw.refresh(): - for pval in applied_ptransform.outputs.values(): + for pval in list(applied_ptransform.outputs.values()): if isinstance(pval, pvalue.DoOutputsTuple): pvals = (v for v in pval) else: @@ -145,7 +144,7 @@ def _refresh_watermarks(self, applied_ptransform): def extract_fired_timers(self): all_timers = [] - for applied_ptransform, tw in self._transform_to_watermarks.iteritems(): + for applied_ptransform, tw in self._transform_to_watermarks.items(): fired_timers = tw.extract_fired_timers() if fired_timers: all_timers.append((applied_ptransform, fired_timers)) @@ -189,7 +188,7 @@ def output_watermark(self): def hold(self, keyed_earliest_holds): with self._lock: - for key, hold_value in keyed_earliest_holds.iteritems(): + for key, hold_value in keyed_earliest_holds.items(): self._keyed_earliest_holds[key] = hold_value if (hold_value is None or hold_value == WatermarkManager.WATERMARK_POS_INF): @@ -233,7 +232,7 @@ def refresh(self): self._input_watermark = max(self._input_watermark, min(pending_holder, producer_watermark)) earliest_hold = WatermarkManager.WATERMARK_POS_INF - for hold in self._keyed_earliest_holds.values(): + for hold in list(self._keyed_earliest_holds.values()): if hold < earliest_hold: earliest_hold = hold new_output_watermark = min(self._input_watermark, earliest_hold) @@ -252,7 +251,7 @@ def extract_fired_timers(self): return False fired_timers = [] - for encoded_key, state in self._keyed_states.iteritems(): + for encoded_key, state in self._keyed_states.items(): timers = state.get_timers(watermark=self._input_watermark) for expired in timers: window, (name, time_domain, timestamp) = expired diff --git a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py index 247ce1f0e195..b94874cf27c4 100644 --- a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py +++ b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py @@ -21,15 +21,14 @@ import logging import random import string +from builtins import range import grpc from apache_beam.portability.api import beam_job_api_pb2 from apache_beam.runners.job import utils as job_utils from apache_beam.runners.job.manager import DockerRPCManager -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineRunner - +from apache_beam.runners.runner import PipelineResult, PipelineRunner __all__ = ['PythonRPCDirectRunner'] @@ -60,7 +59,7 @@ def run(self, pipeline): # Submit the job to the RPC co-process jobName = ('Job-' + ''.join(random.choice(string.ascii_uppercase) for _ in range(6))) - options = {k: v for k, v in pipeline._options.get_all_options().iteritems() + options = {k: v for k, v in pipeline._options.get_all_options().items() if v is not None} try: diff --git a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py index bae25a4d07b1..8e56f9f59f63 100644 --- a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py +++ b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py @@ -23,10 +23,9 @@ from concurrent import futures import grpc -from apache_beam.portability.api import beam_job_api_pb2 -from apache_beam.portability.api import beam_job_api_pb2_grpc -from apache_beam.pipeline import Pipeline from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.pipeline import Pipeline +from apache_beam.portability.api import beam_job_api_pb2, beam_job_api_pb2_grpc from apache_beam.runners.runner import PipelineState _ONE_DAY_IN_SECONDS = 60 * 60 * 24 diff --git a/sdks/python/apache_beam/runners/job/manager.py b/sdks/python/apache_beam/runners/job/manager.py index 4d88a1189f23..49f9cbf448f8 100644 --- a/sdks/python/apache_beam/runners/job/manager.py +++ b/sdks/python/apache_beam/runners/job/manager.py @@ -21,6 +21,7 @@ import logging import subprocess import time +from builtins import object import grpc diff --git a/sdks/python/apache_beam/runners/job/utils.py b/sdks/python/apache_beam/runners/job/utils.py index 84c727fb4ebc..80b5e451f2d0 100644 --- a/sdks/python/apache_beam/runners/job/utils.py +++ b/sdks/python/apache_beam/runners/job/utils.py @@ -20,8 +20,7 @@ import json -from google.protobuf import json_format -from google.protobuf import struct_pb2 +from google.protobuf import json_format, struct_pb2 def dict_to_struct(dict_obj): diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index 42d7f5df14a8..a98a5588c538 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -21,11 +21,10 @@ """ -from apache_beam import pipeline -from apache_beam import pvalue -from apache_beam import coders -from apache_beam.portability.api import beam_fn_api_pb2 -from apache_beam.portability.api import beam_runner_api_pb2 +from builtins import object + +from apache_beam import coders, pipeline, pvalue +from apache_beam.portability.api import beam_fn_api_pb2, beam_runner_api_pb2 from apache_beam.transforms import core @@ -49,7 +48,7 @@ def _unique_ref(self, obj=None, label=None): self._obj_type.__name__, label or type(obj).__name__, self._counter) def populate_map(self, proto_map): - for id, proto in self._id_to_proto.items(): + for id, proto in list(self._id_to_proto.items()): proto_map[id].CopyFrom(proto) def get_id(self, obj, label=None): @@ -90,10 +89,10 @@ class PipelineContext(object): def __init__(self, proto=None): if isinstance(proto, beam_fn_api_pb2.ProcessBundleDescriptor): proto = beam_runner_api_pb2.Components( - coders=dict(proto.coders.items()), - windowing_strategies=dict(proto.windowing_strategies.items()), - environments=dict(proto.environments.items())) - for name, cls in self._COMPONENT_TYPES.items(): + coders=dict(list(proto.coders.items())), + windowing_strategies=dict(list(proto.windowing_strategies.items())), + environments=dict(list(proto.environments.items()))) + for name, cls in list(self._COMPONENT_TYPES.items()): setattr( self, name, _PipelineContextMap( self, cls, getattr(proto, name, None))) diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index 7c0c06fe1110..146810cc8d58 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -21,33 +21,32 @@ import collections import copy import logging -import Queue as queue +import queue as queue import threading - +from builtins import object from concurrent import futures -from google.protobuf import wrappers_pb2 + import grpc +from future import standard_library +from google.protobuf import wrappers_pb2 import apache_beam as beam # pylint: disable=ungrouped-imports -from apache_beam.coders import registry -from apache_beam.coders import WindowedValueCoder -from apache_beam.coders.coder_impl import create_InputStream -from apache_beam.coders.coder_impl import create_OutputStream +from apache_beam.coders import WindowedValueCoder, registry +from apache_beam.coders.coder_impl import (create_InputStream, + create_OutputStream) from apache_beam.internal import pickler from apache_beam.io import iobase from apache_beam.metrics.execution import MetricsEnvironment -from apache_beam.portability.api import beam_fn_api_pb2 -from apache_beam.portability.api import beam_runner_api_pb2 +from apache_beam.portability.api import beam_fn_api_pb2, beam_runner_api_pb2 from apache_beam.runners import pipeline_context from apache_beam.runners.portability import maptask_executor_runner from apache_beam.runners.runner import PipelineState -from apache_beam.runners.worker import bundle_processor -from apache_beam.runners.worker import data_plane -from apache_beam.runners.worker import operation_specs -from apache_beam.runners.worker import sdk_worker +from apache_beam.runners.worker import (bundle_processor, data_plane, + operation_specs, sdk_worker) from apache_beam.transforms.window import GlobalWindows -from apache_beam.utils import proto_utils -from apache_beam.utils import urns +from apache_beam.utils import proto_utils, urns + +standard_library.install_aliases() # This module is experimental. No backwards-compatibility guarantees. @@ -136,7 +135,7 @@ def append(self, elements_data): def __iter__(self): output_stream = create_OutputStream() - for encoded_key, values in self._table.items(): + for encoded_key, values in list(self._table.items()): key = self._key_coder.decode(encoded_key) self._post_grouped_coder.get_impl().encode_to_stream( GlobalWindows.windowed_value((key, values)), output_stream, True) @@ -231,7 +230,7 @@ def has_as_main_input(self, pcoll): local_side_inputs = payload.side_inputs else: local_side_inputs = {} - for local_id, pipeline_id in transform.inputs.items(): + for local_id, pipeline_id in list(transform.inputs.items()): if pcoll == pipeline_id and local_id not in local_side_inputs: return True @@ -240,7 +239,7 @@ def deduplicate_read(self): new_transforms = [] for transform in self.transforms: if transform.spec.urn == bundle_processor.DATA_INPUT_URN: - pcoll = only_element(transform.outputs.items())[1] + pcoll = only_element(list(transform.outputs.items()))[1] if pcoll in seen_pcolls: continue seen_pcolls.add(pcoll) @@ -266,8 +265,8 @@ def expand_gbk(stages): spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, any_param=proto_utils.pack_Any( - wrappers_pb2.BytesValue(value=param)), - payload=param))], + wrappers_pb2.BytesValue(value=param.encode())), + payload=param.encode()))], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) yield gbk_write @@ -280,8 +279,8 @@ def expand_gbk(stages): spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, any_param=proto_utils.pack_Any( - wrappers_pb2.BytesValue(value=param)), - payload=param))], + wrappers_pb2.BytesValue(value=param.encode())), + payload=param.encode()))], downstream_side_inputs=frozenset(), must_follow=union(frozenset([gbk_write]), stage.must_follow)) else: @@ -302,10 +301,10 @@ def sink_flattens(stages): if transform.spec.urn == urns.FLATTEN_TRANSFORM: # This is used later to correlate the read and writes. param = str("materialize:%s" % transform.unique_name) - output_pcoll_id, = transform.outputs.values() + output_pcoll_id, = list(transform.outputs.values()) output_coder_id = pcollections[output_pcoll_id].coder_id flatten_writes = [] - for local_in, pcoll_in in transform.inputs.items(): + for local_in, pcoll_in in list(transform.inputs.items()): if pcollections[pcoll_in].coder_id != output_coder_id: # Flatten inputs must all be written with the same coder as is @@ -339,8 +338,8 @@ def sink_flattens(stages): urn=bundle_processor.DATA_OUTPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=param)), - payload=param))], + value=param.encode())), + payload=param.encode()))], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) flatten_writes.append(flatten_write) @@ -355,8 +354,8 @@ def sink_flattens(stages): urn=bundle_processor.DATA_INPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=param)), - payload=param))], + value=param.encode())), + payload=param.encode()))], downstream_side_inputs=frozenset(), must_follow=union(frozenset(flatten_writes), stage.must_follow)) @@ -382,7 +381,7 @@ def annotate_downstream_side_inputs(stages): all_side_inputs = set() for stage in stages: for transform in stage.transforms: - for input in transform.inputs.values(): + for input in list(transform.inputs.values()): consumers[input].append(stage) for si in stage.side_inputs(): all_side_inputs.add(si) @@ -394,7 +393,7 @@ def compute_downstream_side_inputs(stage): if stage not in downstream_side_inputs_by_stage: downstream_side_inputs = frozenset() for transform in stage.transforms: - for output in transform.outputs.values(): + for output in list(transform.outputs.values()): if output in all_side_inputs: downstream_side_inputs = union(downstream_side_inputs, output) for consumer in consumers[output]: @@ -435,16 +434,16 @@ def fuse(producer, consumer): # First record the producers and consumers of each PCollection. for stage in stages: for transform in stage.transforms: - for input in transform.inputs.values(): + for input in list(transform.inputs.values()): consumers_by_pcoll[input].append(stage) - for output in transform.outputs.values(): + for output in list(transform.outputs.values()): producers_by_pcoll[output] = stage logging.debug('consumers\n%s', consumers_by_pcoll) logging.debug('producers\n%s', producers_by_pcoll) # Now try to fuse away all pcollections. - for pcoll, producer in producers_by_pcoll.items(): + for pcoll, producer in list(producers_by_pcoll.items()): pcoll_as_param = str("materialize:%s" % pcoll) write_pcoll = None for consumer in consumers_by_pcoll[pcoll]: @@ -467,8 +466,8 @@ def fuse(producer, consumer): urn=bundle_processor.DATA_OUTPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=pcoll_as_param)), - payload=pcoll_as_param))]) + value=pcoll_as_param.encode())), + payload=pcoll_as_param.encode()))]) fuse(producer, write_pcoll) if consumer.has_as_main_input(pcoll): read_pcoll = Stage( @@ -480,15 +479,16 @@ def fuse(producer, consumer): urn=bundle_processor.DATA_INPUT_URN, any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=pcoll_as_param)), - payload=pcoll_as_param))], + value=pcoll_as_param.encode())), + payload=pcoll_as_param.encode()))], must_follow={write_pcoll}) fuse(read_pcoll, consumer) # Everything that was originally a stage or a replacement, but wasn't # replaced, should be in the final graph. - final_stages = frozenset(stages).union(replacements.values()).difference( - replacements.keys()) + final_stages = frozenset(stages).union( + list(replacements.values())).difference( + list(replacements.keys())) for stage in final_stages: # Update all references to their final values before throwing @@ -521,7 +521,7 @@ def process(stage): # Reify coders. # TODO(BEAM-2717): Remove once Coders are already in proto. coders = pipeline_context.PipelineContext(pipeline_components).coders - for pcoll in pipeline_components.pcollections.values(): + for pcoll in list(pipeline_components.pcollections.values()): if pcoll.coder_id not in coders: window_coder = coders[ pipeline_components.windowing_strategies[ @@ -606,11 +606,11 @@ def extract_endpoints(stage): id=self._next_uid(), transforms={transform.unique_name: transform for transform in stage.transforms}, - pcollections=dict(pipeline_components.pcollections.items()), - coders=dict(pipeline_components.coders.items()), + pcollections=dict(list(pipeline_components.pcollections.items())), + coders=dict(list(pipeline_components.coders.items())), windowing_strategies=dict( - pipeline_components.windowing_strategies.items()), - environments=dict(pipeline_components.environments.items())) + list(pipeline_components.windowing_strategies.items())), + environments=dict(list(pipeline_components.environments.items()))) process_bundle_registration = beam_fn_api_pb2.InstructionRequest( instruction_id=self._next_uid(), @@ -624,7 +624,7 @@ def extract_endpoints(stage): process_bundle_descriptor.id)) # Write all the input data to the channel. - for (transform_id, name), pcoll_id in data_input.items(): + for (transform_id, name), pcoll_id in list(data_input.items()): data_out = controller.data_plane_handler.output_stream( process_bundle.instruction_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) @@ -648,7 +648,7 @@ def extract_endpoints(stage): expected_targets = [ beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, name=output_name) - for (transform_id, output_name), _ in data_output.items()] + for (transform_id, output_name), _ in list(data_output.items())] for output in controller.data_plane_handler.input_elements( process_bundle.instruction_id, expected_targets): target_tuple = ( @@ -664,8 +664,8 @@ def extract_endpoints(stage): original_gbk_transform = pcoll_id.split(':', 1)[1] transform_proto = pipeline_components.transforms[ original_gbk_transform] - input_pcoll = only_element(transform_proto.inputs.values()) - output_pcoll = only_element(transform_proto.outputs.values()) + input_pcoll = only_element(list(transform_proto.inputs.values())) + output_pcoll = only_element(list(transform_proto.outputs.values())) pre_gbk_coder = coders[ pipeline_components.pcollections[input_pcoll].coder_id] post_gbk_coder = coders[ @@ -686,7 +686,7 @@ def _map_task_registration(self, map_task, state_handler, input_data, side_input_data, runner_sinks, process_bundle_descriptor = ( self._map_task_to_protos(map_task, data_operation_spec)) # Side inputs will be accessed over the state API. - for key, elements_data in side_input_data.items(): + for key, elements_data in list(side_input_data.items()): state_key = beam_fn_api_pb2.StateKey.MultimapSideInput(key=key) state_handler.Clear(state_key) state_handler.Append(state_key, [elements_data]) @@ -735,7 +735,7 @@ def get_outputs(op_ix): if isinstance(operation, operation_specs.WorkerInMemoryWrite): # Write this data back to the runner. - target_name = only_element(get_inputs(operation).keys()) + target_name = only_element(list(get_inputs(operation).keys())) runner_sinks[(transform_id, target_name)] = operation transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, @@ -749,7 +749,7 @@ def get_outputs(op_ix): maptask_executor_runner.InMemorySource) and isinstance(operation.source.source.default_output_coder(), WindowedValueCoder)): - target_name = only_element(get_outputs(op_ix).keys()) + target_name = only_element(list(get_outputs(op_ix).keys())) input_data[(transform_id, target_name)] = self._reencode_elements( operation.source.source.read(None), operation.source.source.default_output_coder()) @@ -813,7 +813,7 @@ def get_outputs(op_ix): unique_name=name, coder_id=context.coders.get_id( map_task[op_id][1].output_coders[out_id])) - for (op_id, out_id), name in used_pcollections.items() + for (op_id, out_id), name in list(used_pcollections.items()) } # Must follow creation of pcollection_protos to capture used coders. context_proto = context.to_runner_api() @@ -821,9 +821,9 @@ def get_outputs(op_ix): id=self._next_uid(), transforms=transform_protos, pcollections=pcollection_protos, - coders=dict(context_proto.coders.items()), + coders=dict(list(context_proto.coders.items())), windowing_strategies=dict(context_proto.windowing_strategies.items()), - environments=dict(context_proto.environments.items())) + environments=dict(list(context_proto.environments.items()))) return input_data, side_input_data, runner_sinks, process_bundle_descriptor def _run_map_task( @@ -838,7 +838,7 @@ def _run_map_task( process_bundle_descriptor_reference=registration.register. process_bundle_descriptor[0].id)) - for (transform_id, name), elements in input_data.items(): + for (transform_id, name), elements in list(input_data.items()): data_out = data_plane_handler.output_stream( process_bundle.instruction_id, beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, name=name)) @@ -854,7 +854,7 @@ def _run_map_task( expected_targets = [ beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, name=output_name) - for (transform_id, output_name), _ in sinks.items()] + for (transform_id, output_name), _ in list(sinks.items())] for output in data_plane_handler.input_elements( process_bundle.instruction_id, expected_targets): target_tuple = ( @@ -934,9 +934,11 @@ def __init__(self): self.data_plane_handler.inverse())) def push(self, request): - logging.info('CONTROL REQUEST %s', request) + request_str = str(request) + logging.info('CONTROL REQUEST %s', request_str) response = self.worker.do_instruction(request) - logging.info('CONTROL RESPONSE %s', response) + response_str = str(response) + logging.info('CONTROL RESPONSE %s', response_str) self._responses.append(response) def pull(self): diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py index ba219547e6cf..ef7660518bd5 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py @@ -19,10 +19,9 @@ import unittest import apache_beam as beam -from apache_beam.runners.portability import fn_api_runner -from apache_beam.runners.portability import maptask_executor_runner_test -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to +from apache_beam.runners.portability import (fn_api_runner, + maptask_executor_runner_test) +from apache_beam.testing.util import assert_that, equal_to class FnApiRunnerTest( diff --git a/sdks/python/apache_beam/runners/portability/maptask_executor_runner.py b/sdks/python/apache_beam/runners/portability/maptask_executor_runner.py index ddfc4ccc43f1..39bcb3de9d0a 100644 --- a/sdks/python/apache_beam/runners/portability/maptask_executor_runner.py +++ b/sdks/python/apache_beam/runners/portability/maptask_executor_runner.py @@ -21,6 +21,7 @@ import collections import logging import time +from builtins import object, zip import apache_beam as beam from apache_beam.internal import pickler @@ -28,21 +29,21 @@ from apache_beam.metrics.execution import MetricsEnvironment from apache_beam.options import pipeline_options from apache_beam.runners import DataflowRunner -from apache_beam.runners.dataflow.internal.dependency import _dependency_file_copy +from apache_beam.runners.dataflow.internal.dependency import \ + _dependency_file_copy from apache_beam.runners.dataflow.internal.names import PropertyNames from apache_beam.runners.dataflow.native_io.iobase import NativeSource -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineRunner -from apache_beam.runners.runner import PipelineState -from apache_beam.runners.worker import operation_specs -from apache_beam.runners.worker import operations +from apache_beam.runners.runner import (PipelineResult, PipelineRunner, + PipelineState) +from apache_beam.runners.worker import operation_specs, operations +from apache_beam.typehints import typehints +from apache_beam.utils import profiler +from apache_beam.utils.counters import CounterFactory + try: from apache_beam.runners.worker import statesampler except ImportError: from apache_beam.runners.worker import statesampler_fake as statesampler -from apache_beam.typehints import typehints -from apache_beam.utils import profiler -from apache_beam.utils.counters import CounterFactory # This module is experimental. No backwards-compatibility guarantees. @@ -90,7 +91,7 @@ def compute_depth(x): memoized[x] = 1 + max([-1] + [compute_depth(y) for y in deps[x]]) return memoized[x] - return {x: compute_depth(x) for x in deps.keys()} + return {x: compute_depth(x) for x in list(deps.keys())} map_task_depths = compute_depth_map(self.dependencies) ordered_map_tasks = sorted((map_task_depths.get(ix, -1), map_task) @@ -119,7 +120,7 @@ def execute_map_tasks(self, ordered_map_tasks): for ix, (_, map_task) in enumerate(ordered_map_tasks): logging.info('Running %s', map_task) t = time.time() - stage_names, all_operations = zip(*map_task) + stage_names, all_operations = list(zip(*map_task)) # TODO(robertwb): The DataflowRunner worker receives system step names # (e.g. "s3") that are used to label the output msec counters. We use the # operation names here, but this is not the same scheme used by the @@ -394,7 +395,7 @@ def append(self, pair): def freeze(self): if not self.frozen: self._encoded_elements = [self.grouped_coder.encode(kv) - for kv in self.elements.iteritems()] + for kv in self.elements.items()] self.frozen = True return self._encoded_elements @@ -417,7 +418,7 @@ def __iter__(self): def __len__(self): return len(self.buffer.freeze()) - def __nonzero__(self): + def __bool__(self): return True @@ -434,7 +435,7 @@ def expand(self, input): def to_accumulator(v): return self.combine_fn.add_input( self.combine_fn.create_accumulator(), v) - return input | beam.Map(lambda (k, v): (k, to_accumulator(v))) + return input | beam.Map(lambda k_v: (k_v[0], to_accumulator(k_v[1]))) class MergeAccumulators(beam.PTransform): @@ -448,7 +449,11 @@ def expand(self, input): return beam.pvalue.PCollection(input.pipeline) else: merge_accumulators = self.combine_fn.merge_accumulators - return input | beam.Map(lambda (k, vs): (k, merge_accumulators(vs))) + + def combine_local(k_vs): + return (k_vs[0], merge_accumulators(k_vs[1])) + + return input | beam.Map(combine_local) class ExtractOutputs(beam.PTransform): @@ -462,7 +467,7 @@ def expand(self, input): return beam.pvalue.PCollection(input.pipeline) else: extract_output = self.combine_fn.extract_output - return input | beam.Map(lambda (k, v): (k, extract_output(v))) + return input | beam.Map(lambda k_v1: (k_v1[0], extract_output(k_v1[1]))) class WorkerRunnerResult(PipelineResult): diff --git a/sdks/python/apache_beam/runners/portability/maptask_executor_runner_test.py b/sdks/python/apache_beam/runners/portability/maptask_executor_runner_test.py index b7ba15a502aa..0d9861566c6c 100644 --- a/sdks/python/apache_beam/runners/portability/maptask_executor_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/maptask_executor_runner_test.py @@ -21,18 +21,13 @@ import unittest import apache_beam as beam - from apache_beam.metrics import Metrics -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.execution import MetricsEnvironment +from apache_beam.metrics.execution import MetricKey, MetricsEnvironment from apache_beam.metrics.metricbase import MetricName - from apache_beam.pvalue import AsList -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import BeamAssertException -from apache_beam.testing.util import equal_to -from apache_beam.transforms.window import TimestampedValue from apache_beam.runners.portability import maptask_executor_runner +from apache_beam.testing.util import BeamAssertException, assert_that, equal_to +from apache_beam.transforms.window import TimestampedValue class MapTaskExecutorRunnerTest(unittest.TestCase): @@ -88,7 +83,7 @@ def process(self, element): counter_updates = [{'key': key, 'value': val} for container in p.runner.metrics_containers() for key, val in - container.get_updates().counters.items()] + list(container.get_updates().counters.items())] counter_values = [update['value'] for update in counter_updates] counter_keys = [update['key'] for update in counter_updates] assert_that(res, equal_to([1, 2, 3])) @@ -154,7 +149,7 @@ def cross_product(elem, sides): derived = ((pcoll,) | beam.Flatten() | beam.Map(lambda x: (x, x)) | beam.GroupByKey() - | 'Unkey' >> beam.Map(lambda (x, _): x)) + | 'Unkey' >> beam.Map(lambda x__: x__[0])) assert_that( pcoll | beam.FlatMap(cross_product, AsList(derived)), equal_to([('a', 'a'), ('a', 'b'), ('b', 'a'), ('b', 'b')])) @@ -164,7 +159,7 @@ def test_group_by_key(self): res = (p | beam.Create([('a', 1), ('a', 2), ('b', 3)]) | beam.GroupByKey() - | beam.Map(lambda (k, vs): (k, sorted(vs)))) + | beam.Map(lambda k_vs: (k_vs[0], sorted(k_vs[1])))) assert_that(res, equal_to([('a', [1, 2]), ('b', [3])])) def test_flatten(self): @@ -201,7 +196,7 @@ def test_windowing(self): | beam.Map(lambda t: TimestampedValue(('k', t), t)) | beam.WindowInto(beam.transforms.window.Sessions(10)) | beam.GroupByKey() - | beam.Map(lambda (k, vs): (k, sorted(vs)))) + | beam.Map(lambda k_vs1: (k_vs1[0], sorted(k_vs1[1])))) assert_that(res, equal_to([('k', [1, 2]), ('k', [100, 101, 102])])) def test_errors(self): diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index 43ee27bc298f..8d8ae6f022c2 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -24,7 +24,7 @@ import shelve import shutil import tempfile - +from builtins import object __all__ = ['PipelineRunner', 'PipelineState', 'PipelineResult'] diff --git a/sdks/python/apache_beam/runners/runner_test.py b/sdks/python/apache_beam/runners/runner_test.py index fa80b1c9555c..1b3b1b4f69f3 100644 --- a/sdks/python/apache_beam/runners/runner_test.py +++ b/sdks/python/apache_beam/runners/runner_test.py @@ -28,17 +28,13 @@ import apache_beam as beam import apache_beam.transforms as ptransform -from apache_beam.metrics.cells import DistributionData -from apache_beam.metrics.cells import DistributionResult -from apache_beam.metrics.execution import MetricKey -from apache_beam.metrics.execution import MetricResult +from apache_beam.metrics.cells import DistributionData, DistributionResult +from apache_beam.metrics.execution import MetricKey, MetricResult from apache_beam.metrics.metricbase import MetricName -from apache_beam.pipeline import Pipeline -from apache_beam.runners import DirectRunner -from apache_beam.runners import create_runner -from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.pipeline import Pipeline +from apache_beam.runners import DirectRunner, create_runner +from apache_beam.testing.util import assert_that, equal_to class RunnerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 16c888c63c60..d5ea7b26cafd 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -17,31 +17,25 @@ """SDK harness for executing Python Fns via the Fn API.""" -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function import base64 import collections import json import logging +from builtins import object from google.protobuf import wrappers_pb2 import apache_beam as beam -from apache_beam.coders import coder_impl -from apache_beam.coders import WindowedValueCoder +from apache_beam.coders import WindowedValueCoder, coder_impl from apache_beam.internal import pickler from apache_beam.io import iobase -from apache_beam.portability.api import beam_fn_api_pb2 -from apache_beam.portability.api import beam_runner_api_pb2 -from apache_beam.runners.dataflow.native_io import iobase as native_iobase +from apache_beam.portability.api import beam_fn_api_pb2, beam_runner_api_pb2 from apache_beam.runners import pipeline_context -from apache_beam.runners.worker import operation_specs -from apache_beam.runners.worker import operations -from apache_beam.utils import counters -from apache_beam.utils import proto_utils -from apache_beam.utils import urns +from apache_beam.runners.dataflow.native_io import iobase as native_iobase +from apache_beam.runners.worker import operation_specs, operations +from apache_beam.utils import counters, proto_utils, urns # This module is experimental. No backwards-compatibility guarantees. @@ -79,7 +73,7 @@ def __init__(self, operation_name, step_name, consumers, counter_factory, # DataInputOperation or a producer of these bytes for a DataOutputOperation. self.target = target self.data_channel = data_channel - for _, consumer_ops in consumers.items(): + for _, consumer_ops in list(consumers.items()): for consumer in consumer_ops: self.add_receiver(consumer, 0) @@ -112,7 +106,7 @@ def __init__(self, operation_name, step_name, consumers, counter_factory, # We must do this manually as we don't have a spec or spec.output_coders. self.receivers = [ operations.ConsumerSet(self.counter_factory, self.step_name, 0, - consumers.itervalues().next(), + next(iter(consumers.values())), self.windowed_coder)] def process(self, windowed_value): @@ -200,8 +194,8 @@ def create_execution_tree(self, descriptor): self.state_handler) pcoll_consumers = collections.defaultdict(list) - for transform_id, transform_proto in descriptor.transforms.items(): - for pcoll_id in transform_proto.inputs.values(): + for transform_id, transform_proto in list(descriptor.transforms.items()): + for pcoll_id in list(transform_proto.inputs.values()): pcoll_consumers[pcoll_id].append(transform_id) @memoize @@ -209,7 +203,7 @@ def get_operation(transform_id): transform_consumers = { tag: [get_operation(op) for op in pcoll_consumers[pcoll_id]] for tag, pcoll_id - in descriptor.transforms[transform_id].outputs.items() + in list(descriptor.transforms[transform_id].outputs.items()) } return transform_factory.create_operation( transform_id, transform_consumers) @@ -298,26 +292,26 @@ def get_coder(self, coder_id): def get_output_coders(self, transform_proto): return { tag: self.get_coder(self.descriptor.pcollections[pcoll_id].coder_id) - for tag, pcoll_id in transform_proto.outputs.items() + for tag, pcoll_id in list(transform_proto.outputs.items()) } def get_only_output_coder(self, transform_proto): - return only_element(self.get_output_coders(transform_proto).values()) + return only_element(list(self.get_output_coders(transform_proto).values())) def get_input_coders(self, transform_proto): return { tag: self.get_coder(self.descriptor.pcollections[pcoll_id].coder_id) - for tag, pcoll_id in transform_proto.inputs.items() + for tag, pcoll_id in list(transform_proto.inputs.items()) } def get_only_input_coder(self, transform_proto): - return only_element(self.get_input_coders(transform_proto).values()) + return only_element(list(self.get_input_coders(transform_proto).values())) # TODO(robertwb): Update all operations to take these in the constructor. @staticmethod def augment_oldstyle_op(op, step_name, consumers, tag_list=None): op.step_name = step_name - for tag, op_consumers in consumers.items(): + for tag, op_consumers in list(consumers.items()): for consumer in op_consumers: op.add_receiver(consumer, tag_list.index(tag) if tag_list else 0) return op @@ -328,7 +322,7 @@ def augment_oldstyle_op(op, step_name, consumers, tag_list=None): def create(factory, transform_id, transform_proto, grpc_port, consumers): target = beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, - name=only_element(transform_proto.outputs.keys())) + name=only_element(list(transform_proto.outputs.keys()))) return DataInputOperation( transform_proto.unique_name, transform_proto.unique_name, @@ -345,7 +339,7 @@ def create(factory, transform_id, transform_proto, grpc_port, consumers): def create(factory, transform_id, transform_proto, grpc_port, consumers): target = beam_fn_api_pb2.Target( primitive_transform_reference=transform_id, - name=only_element(transform_proto.inputs.keys())) + name=only_element(list(transform_proto.inputs.keys()))) return DataOutputOperation( transform_proto.unique_name, transform_proto.unique_name, @@ -451,7 +445,7 @@ def mutate_tag(tag): dofn_data = pickler.loads(serialized_fn) if not dofn_data[-1]: # Windowing not set. - pcoll_id, = transform_proto.inputs.values() + pcoll_id, = list(transform_proto.inputs.values()) windowing = factory.context.windowing_strategies.get_by_id( factory.descriptor.pcollections[pcoll_id].windowing_strategy_id) serialized_fn = pickler.dumps(dofn_data[:-1] + (windowing,)) diff --git a/sdks/python/apache_beam/runners/worker/data_plane.py b/sdks/python/apache_beam/runners/worker/data_plane.py index e713041d7c83..0c3c76d038b3 100644 --- a/sdks/python/apache_beam/runners/worker/data_plane.py +++ b/sdks/python/apache_beam/runners/worker/data_plane.py @@ -17,19 +17,24 @@ """Implementation of DataChannels for communicating across the data plane.""" -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function import abc import collections import logging -import Queue as queue +import queue as queue import threading +from builtins import object, range + +import grpc +from future import standard_library +from future.utils import with_metaclass from apache_beam.coders import coder_impl from apache_beam.portability.api import beam_fn_api_pb2 -import grpc + +standard_library.install_aliases() + # This module is experimental. No backwards-compatibility guarantees. @@ -46,7 +51,7 @@ def close(self): self._close_callback(self.get()) -class DataChannel(object): +class DataChannel(with_metaclass(abc.ABCMeta, object)): """Represents a channel for reading and writing data over the data plane. Read from this channel with the input_elements method:: @@ -65,8 +70,6 @@ class DataChannel(object): data_channel.close() """ - __metaclass__ = abc.ABCMeta - @abc.abstractmethod def input_elements(self, instruction_id, expected_targets): """Returns an iterable of all Element.Data bundles for instruction_id. @@ -240,11 +243,9 @@ def Data(self, elements_iterator, context): yield elements -class DataChannelFactory(object): +class DataChannelFactory(with_metaclass(abc.ABCMeta, object)): """An abstract factory for creating ``DataChannel``.""" - __metaclass__ = abc.ABCMeta - @abc.abstractmethod def create_data_channel(self, remote_grpc_port): """Returns a ``DataChannel`` from the given RemoteGrpcPort.""" @@ -282,7 +283,7 @@ def create_data_channel(self, remote_grpc_port): def close(self): logging.info('Closing all cached grpc data channels.') - for _, channel in self._data_channel_cache.items(): + for _, channel in list(self._data_channel_cache.items()): channel.close() self._data_channel_cache.clear() diff --git a/sdks/python/apache_beam/runners/worker/data_plane_test.py b/sdks/python/apache_beam/runners/worker/data_plane_test.py index 360468a86874..be6a15c25727 100644 --- a/sdks/python/apache_beam/runners/worker/data_plane_test.py +++ b/sdks/python/apache_beam/runners/worker/data_plane_test.py @@ -17,21 +17,22 @@ """Tests for apache_beam.runners.worker.data_plane.""" -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function import logging import sys import threading import unittest - from concurrent import futures + import grpc +from future import standard_library from apache_beam.portability.api import beam_fn_api_pb2 from apache_beam.runners.worker import data_plane +standard_library.install_aliases() + def timeout(timeout_secs): def decorate(fn): diff --git a/sdks/python/apache_beam/runners/worker/log_handler.py b/sdks/python/apache_beam/runners/worker/log_handler.py index b8f635210d2e..91b4e3156dc5 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler.py +++ b/sdks/python/apache_beam/runners/worker/log_handler.py @@ -18,11 +18,17 @@ import logging import math -import Queue as queue +import queue as queue import threading +from builtins import range -from apache_beam.portability.api import beam_fn_api_pb2 import grpc +from future import standard_library + +from apache_beam.portability.api import beam_fn_api_pb2 + +standard_library.install_aliases() + # This module is experimental. No backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/runners/worker/log_handler_test.py b/sdks/python/apache_beam/runners/worker/log_handler_test.py index 2256bb5556f0..9702493a8a79 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler_test.py +++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py @@ -18,8 +18,9 @@ import logging import unittest - +from builtins import range from concurrent import futures + import grpc from apache_beam.portability.api import beam_fn_api_pb2 @@ -99,7 +100,7 @@ def _create_test(name, num_logs): if __name__ == '__main__': - for test_name, num_logs_entries in data.iteritems(): + for test_name, num_logs_entries in data.items(): _create_test(test_name, num_logs_entries) unittest.main() diff --git a/sdks/python/apache_beam/runners/worker/logger_test.py b/sdks/python/apache_beam/runners/worker/logger_test.py index cf3f69292826..b5426625fc62 100644 --- a/sdks/python/apache_beam/runners/worker/logger_test.py +++ b/sdks/python/apache_beam/runners/worker/logger_test.py @@ -22,9 +22,14 @@ import sys import threading import unittest +from builtins import object + +from future import standard_library from apache_beam.runners.worker import logger +standard_library.install_aliases() + class PerThreadLoggingContextTest(unittest.TestCase): @@ -83,7 +88,7 @@ def create_log_record(self, **kwargs): class Record(object): def __init__(self, **kwargs): - for k, v in kwargs.iteritems(): + for k, v in kwargs.items(): setattr(self, k, v) return Record(**kwargs) diff --git a/sdks/python/apache_beam/runners/worker/opcounters.py b/sdks/python/apache_beam/runners/worker/opcounters.py index 2bb15fa7ee4b..f48cc6fa4435 100644 --- a/sdks/python/apache_beam/runners/worker/opcounters.py +++ b/sdks/python/apache_beam/runners/worker/opcounters.py @@ -19,9 +19,13 @@ """Counters collect the progress of the Worker for reporting to the service.""" -from __future__ import absolute_import +from __future__ import absolute_import, division + import math import random +from builtins import hex, object + +from past.utils import old_div from apache_beam.utils.counters import Counter @@ -100,7 +104,9 @@ def update_collect(self): def _compute_next_sample(self, i): # https://en.wikipedia.org/wiki/Reservoir_sampling#Fast_Approximation - gap = math.log(1.0 - random.random()) / math.log(1.0 - 10.0/i) + gap = old_div( + math.log(1.0 - random.random()), + math.log(1.0 - old_div(10.0, i))) return i + math.floor(gap) def _should_sample(self): diff --git a/sdks/python/apache_beam/runners/worker/opcounters_test.py b/sdks/python/apache_beam/runners/worker/opcounters_test.py index 74561b81593e..def6be7f9c7d 100644 --- a/sdks/python/apache_beam/runners/worker/opcounters_test.py +++ b/sdks/python/apache_beam/runners/worker/opcounters_test.py @@ -1,3 +1,18 @@ +from __future__ import division + +import logging +import math +import random +import unittest +from builtins import object, range + +from past.utils import old_div + +from apache_beam import coders +from apache_beam.runners.worker.opcounters import OperationCounters +from apache_beam.transforms.window import GlobalWindows +from apache_beam.utils.counters import CounterFactory + # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -15,22 +30,12 @@ # limitations under the License. # -import logging -import math -import random -import unittest - -from apache_beam import coders -from apache_beam.runners.worker.opcounters import OperationCounters -from apache_beam.transforms.window import GlobalWindows -from apache_beam.utils.counters import CounterFactory - # Classes to test that we can handle a variety of objects. # These have to be at top level so the pickler can find them. -class OldClassThatDoesNotImplementLen: # pylint: disable=old-style-class +class OldClassThatDoesNotImplementLen(object): # pylint: disable=old-style-class def __init__(self): pass @@ -104,11 +109,11 @@ def test_update_multiple(self): value = GlobalWindows.windowed_value('defghij') opcounts.update_from(value) total_size += coder.estimate_size(value) - self.verify_counters(opcounts, 2, float(total_size) / 2) + self.verify_counters(opcounts, 2, old_div(float(total_size), 2)) value = GlobalWindows.windowed_value('klmnop') opcounts.update_from(value) total_size += coder.estimate_size(value) - self.verify_counters(opcounts, 3, float(total_size) / 3) + self.verify_counters(opcounts, 3, old_div(float(total_size), 3)) def test_should_sample(self): # Order of magnitude more buckets than highest constant in code under test. @@ -121,27 +126,27 @@ def test_should_sample(self): total_runs = 10 * len(buckets) # Fill the buckets. - for _ in xrange(total_runs): + for _ in range(total_runs): opcounts = OperationCounters(CounterFactory(), 'some-name', coders.PickleCoder(), 0) - for i in xrange(len(buckets)): + for i in range(len(buckets)): if opcounts.should_sample(): buckets[i] += 1 # Look at the buckets to see if they are likely. - for i in xrange(10): + for i in range(10): self.assertEqual(total_runs, buckets[i]) - for i in xrange(10, len(buckets)): + for i in range(10, len(buckets)): self.assertTrue(buckets[i] > 7 * total_runs / i, 'i=%d, buckets[i]=%d, expected=%d, ratio=%f' % ( i, buckets[i], 10 * total_runs / i, - buckets[i] / (10.0 * total_runs / i))) + old_div(buckets[i], (10.0 * total_runs / i)))) self.assertTrue(buckets[i] < 14 * total_runs / i, 'i=%d, buckets[i]=%d, expected=%d, ratio=%f' % ( i, buckets[i], 10 * total_runs / i, - buckets[i] / (10.0 * total_runs / i))) + old_div(buckets[i], (10.0 * total_runs / i)))) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/runners/worker/operation_specs.py b/sdks/python/apache_beam/runners/worker/operation_specs.py index bdafbeaf44ad..06147eaab641 100644 --- a/sdks/python/apache_beam/runners/worker/operation_specs.py +++ b/sdks/python/apache_beam/runners/worker/operation_specs.py @@ -22,6 +22,7 @@ """ import collections +from builtins import object from apache_beam import coders @@ -55,7 +56,7 @@ def worker_printable_fields(workerproto): return ['%s=%s' % (name, value) # _asdict is the only way and cannot subclass this generated class # pylint: disable=protected-access - for name, value in workerproto._asdict().iteritems() + for name, value in workerproto._asdict().items() # want to output value 0 but not None nor [] if (value or value == 0) and name not in diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py index c4f945bf2b9d..fd655921f9a4 100644 --- a/sdks/python/apache_beam/runners/worker/operations.py +++ b/sdks/python/apache_beam/runners/worker/operations.py @@ -20,26 +20,23 @@ """Worker operations executor.""" import collections -import itertools import logging +from builtins import object from apache_beam import pvalue from apache_beam.internal import pickler from apache_beam.io import iobase -from apache_beam.metrics.execution import MetricsContainer -from apache_beam.metrics.execution import ScopedMetricsContainer +from apache_beam.metrics.execution import (MetricsContainer, + ScopedMetricsContainer) from apache_beam.runners import common from apache_beam.runners.common import Receiver from apache_beam.runners.dataflow.internal.names import PropertyNames -from apache_beam.runners.worker import logger -from apache_beam.runners.worker import opcounters -from apache_beam.runners.worker import operation_specs -from apache_beam.runners.worker import sideinputs -from apache_beam.transforms import combiners -from apache_beam.transforms import core +from apache_beam.runners.worker import (logger, opcounters, operation_specs, + sideinputs) from apache_beam.transforms import sideinputs as apache_sideinputs -from apache_beam.transforms.combiners import curry_combine_fn -from apache_beam.transforms.combiners import PhasedCombineFnExecutor +from apache_beam.transforms import combiners, core +from apache_beam.transforms.combiners import (PhasedCombineFnExecutor, + curry_combine_fn) from apache_beam.transforms.window import GlobalWindows from apache_beam.utils.windowed_value import WindowedValue @@ -273,8 +270,8 @@ def _read_side_inputs(self, tags_and_types): # while the variable has the value assigned by the current iteration of # the for loop. # pylint: disable=cell-var-from-loop - for si in itertools.ifilter( - lambda o: o.tag == side_tag, self.spec.side_inputs): + filtered_tags = [o for o in self.spec.side_inputs if o.tag == side_tag] + for si in filtered_tags: if not isinstance(si, operation_specs.WorkerSideInputSource): raise NotImplementedError('Unknown side input type: %r' % si) sources.append(si.source) @@ -434,7 +431,7 @@ def __init__(self, operation_name, spec, counter_factory, state_sampler): fn, args, kwargs = pickler.loads(self.spec.combine_fn)[:3] self.combine_fn = curry_combine_fn(fn, args, kwargs) if (getattr(fn.add_input, 'im_func', None) - is core.CombineFn.add_input.im_func): + is core.CombineFn.add_input.__func__): # Old versions of the SDK have CombineFns that don't implement add_input. self.combine_fn_add_input = ( lambda a, e: self.combine_fn.add_inputs(a, [e])) @@ -467,7 +464,7 @@ def process(self, wkv): target = self.key_count * 9 // 10 old_wkeys = [] # TODO(robertwb): Use an LRU cache? - for old_wkey, old_wvalue in self.table.iteritems(): + for old_wkey, old_wvalue in self.table.items(): old_wkeys.append(old_wkey) # Can't mutate while iterating. self.output_key(old_wkey, old_wvalue[0]) self.key_count -= 1 @@ -482,7 +479,7 @@ def process(self, wkv): entry[0] = self.combine_fn_add_input(entry[0], value) def finish(self): - for wkey, value in self.table.iteritems(): + for wkey, value in self.table.items(): self.output_key(wkey, value[0]) self.table = {} self.key_count = 0 diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py index 6a236802b9a6..e77b3ed969d6 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py @@ -17,18 +17,20 @@ """SDK harness for executing Python Fns via the Fn API.""" -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function import logging -import Queue as queue +import queue as queue import threading import traceback +from builtins import object + +from future import standard_library from apache_beam.portability.api import beam_fn_api_pb2 -from apache_beam.runners.worker import bundle_processor -from apache_beam.runners.worker import data_plane +from apache_beam.runners.worker import bundle_processor, data_plane + +standard_library.install_aliases() class SdkHarness(object): diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py index dc72a5ff4fae..ffb2ca04bedd 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py @@ -17,18 +17,18 @@ """Tests for apache_beam.runners.worker.sdk_worker.""" -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function import logging import unittest - +# We don't import str here since it causes issues and this +# is a test not internal code. +from builtins import range from concurrent import futures + import grpc -from apache_beam.portability.api import beam_fn_api_pb2 -from apache_beam.portability.api import beam_runner_api_pb2 +from apache_beam.portability.api import beam_fn_api_pb2, beam_runner_api_pb2 from apache_beam.runners.worker import sdk_worker @@ -62,12 +62,16 @@ def Control(self, response_iterator, context): class SdkWorkerTest(unittest.TestCase): def test_fn_registration(self): - process_bundle_descriptors = [ - beam_fn_api_pb2.ProcessBundleDescriptor( - id=str(100+ix), - transforms={ - str(ix): beam_runner_api_pb2.PTransform(unique_name=str(ix))}) - for ix in range(4)] + def make_transformer_for_ix(ix): + encoded_ix = str(ix).encode("latin-1") + encoded_ix_100 = str(100+ix).encode("latin-1") + return beam_fn_api_pb2.ProcessBundleDescriptor( + id=encoded_ix_100, + transforms={ + encoded_ix: beam_runner_api_pb2.PTransform( + unique_name=encoded_ix)}) + + process_bundle_descriptors = list(map(make_transformer_for_ix, range(4))) test_controller = BeamFnControlServicer([beam_fn_api_pb2.InstructionRequest( register=beam_fn_api_pb2.RegisterRequest( @@ -81,9 +85,13 @@ def test_fn_registration(self): channel = grpc.insecure_channel("localhost:%s" % test_port) harness = sdk_worker.SdkHarness(channel) harness.run() - self.assertEqual( - harness.worker.fns, - {item.id: item for item in process_bundle_descriptors}) + # We do a funny comparision here because the default formatting in Py2 + # with future gets sad on error. + worker_fns = harness.worker.fns + expected_fns = {item.id: item for item in process_bundle_descriptors} + self.assertEqual(len(worker_fns), len(expected_fns), + "Length of fns did not match") + self.assertEqual(worker_fns, expected_fns) if __name__ == "__main__": diff --git a/sdks/python/apache_beam/runners/worker/sideinputs.py b/sdks/python/apache_beam/runners/worker/sideinputs.py index bdf9f4e71f5e..9375489b6cbf 100644 --- a/sdks/python/apache_beam/runners/worker/sideinputs.py +++ b/sdks/python/apache_beam/runners/worker/sideinputs.py @@ -19,13 +19,19 @@ import collections import logging -import Queue +import queue import threading import traceback +from builtins import object, range + +from future import standard_library from apache_beam.io import iobase from apache_beam.transforms import window +standard_library.install_aliases() + + # This module is experimental. No backwards-compatibility guarantees. @@ -56,13 +62,13 @@ def __init__(self, sources, self.num_reader_threads = min(max_reader_threads, len(self.sources)) # Queue for sources that are to be read. - self.sources_queue = Queue.Queue() + self.sources_queue = queue.Queue() for source in sources: self.sources_queue.put(source) # Queue for elements that have been read. - self.element_queue = Queue.Queue(ELEMENT_QUEUE_SIZE) + self.element_queue = queue.Queue(ELEMENT_QUEUE_SIZE) # Queue for exceptions encountered in reader threads; to be rethrown. - self.reader_exceptions = Queue.Queue() + self.reader_exceptions = queue.Queue() # Whether we have already iterated; this iterable can only be used once. self.already_iterated = False # Whether an error was encountered in any source reader. @@ -105,7 +111,7 @@ def _reader_thread(self): self.element_queue.put(value) else: self.element_queue.put(_globally_windowed(value)) - except Queue.Empty: + except queue.Empty: return except Exception as e: # pylint: disable=broad-except logging.error('Encountered exception in PrefetchingSourceSetIterable ' diff --git a/sdks/python/apache_beam/runners/worker/sideinputs_test.py b/sdks/python/apache_beam/runners/worker/sideinputs_test.py index d243bbe4e6ee..e73f39ddc370 100644 --- a/sdks/python/apache_beam/runners/worker/sideinputs_test.py +++ b/sdks/python/apache_beam/runners/worker/sideinputs_test.py @@ -20,6 +20,7 @@ import logging import time import unittest +from builtins import object, range from apache_beam.runners.worker import sideinputs @@ -67,7 +68,7 @@ def test_single_source_iterator_fn(self): ] iterator_fn = sideinputs.get_iterator_fn_for_sources( sources, max_reader_threads=2) - assert list(strip_windows(iterator_fn())) == range(6) + assert list(strip_windows(iterator_fn())) == list(range(6)) def test_multiple_sources_iterator_fn(self): sources = [ @@ -78,7 +79,7 @@ def test_multiple_sources_iterator_fn(self): ] iterator_fn = sideinputs.get_iterator_fn_for_sources( sources, max_reader_threads=3) - assert sorted(strip_windows(iterator_fn())) == range(11) + assert sorted(strip_windows(iterator_fn())) == list(range(11)) def test_multiple_sources_single_reader_iterator_fn(self): sources = [ @@ -89,7 +90,7 @@ def test_multiple_sources_single_reader_iterator_fn(self): ] iterator_fn = sideinputs.get_iterator_fn_for_sources( sources, max_reader_threads=1) - assert list(strip_windows(iterator_fn())) == range(11) + assert list(strip_windows(iterator_fn())) == list(range(11)) def test_source_iterator_fn_exception(self): class MyException(Exception): @@ -116,7 +117,7 @@ def perpetual_generator(value): with self.assertRaises(MyException): for value in iterator_fn(): seen.add(value.value) - self.assertEqual(sorted(seen), range(5)) + self.assertEqual(sorted(seen), list(range(5))) class EmulatedCollectionsTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fake.py b/sdks/python/apache_beam/runners/worker/statesampler_fake.py index 88ace8c5ae89..aa0fe342e179 100644 --- a/sdks/python/apache_beam/runners/worker/statesampler_fake.py +++ b/sdks/python/apache_beam/runners/worker/statesampler_fake.py @@ -18,6 +18,9 @@ # This module is experimental. No backwards-compatibility guarantees. +from builtins import object + + class StateSampler(object): def __init__(self, *args, **kwargs): diff --git a/sdks/python/apache_beam/runners/worker/statesampler_test.py b/sdks/python/apache_beam/runners/worker/statesampler_test.py index 663cdecdab0c..8569a4fe8acb 100644 --- a/sdks/python/apache_beam/runners/worker/statesampler_test.py +++ b/sdks/python/apache_beam/runners/worker/statesampler_test.py @@ -16,12 +16,15 @@ # """Tests for state sampler.""" +from __future__ import absolute_import, division import logging import time import unittest +from builtins import range from nose.plugins.skip import SkipTest +from past.utils import old_div from apache_beam.utils.counters import CounterFactory @@ -32,7 +35,7 @@ def setUp(self): try: # pylint: disable=global-variable-not-assigned global statesampler - import statesampler + from . import statesampler except ImportError: raise SkipTest('State sampler not compiled.') super(StateSamplerTest, self).setUp() @@ -48,10 +51,10 @@ def test_basic_sampler(self): with sampler.scoped_state('statea'): time.sleep(0.1) with sampler.scoped_state('stateb'): - time.sleep(0.2 / 2) + time.sleep(old_div(0.2, 2)) with sampler.scoped_state('statec'): time.sleep(0.3) - time.sleep(0.2 / 2) + time.sleep(old_div(0.2, 2)) sampler.stop() sampler.commit_counters() diff --git a/sdks/python/apache_beam/testing/pipeline_verifiers.py b/sdks/python/apache_beam/testing/pipeline_verifiers.py index 883343acdab6..c421e259398f 100644 --- a/sdks/python/apache_beam/testing/pipeline_verifiers.py +++ b/sdks/python/apache_beam/testing/pipeline_verifiers.py @@ -32,7 +32,6 @@ from apache_beam.testing import test_utils as utils from apache_beam.utils import retry - __all__ = [ 'PipelineStateMatcher', 'FileChecksumMatcher', diff --git a/sdks/python/apache_beam/testing/pipeline_verifiers_test.py b/sdks/python/apache_beam/testing/pipeline_verifiers_test.py index 15e0a042008b..da8adc9cc259 100644 --- a/sdks/python/apache_beam/testing/pipeline_verifiers_test.py +++ b/sdks/python/apache_beam/testing/pipeline_verifiers_test.py @@ -20,15 +20,15 @@ import logging import tempfile import unittest +from builtins import range from hamcrest import assert_that as hc_assert_that from mock import Mock, patch from apache_beam.io.localfilesystem import LocalFileSystem -from apache_beam.runners.runner import PipelineResult -from apache_beam.runners.runner import PipelineState -from apache_beam.testing.test_utils import patch_retry +from apache_beam.runners.runner import PipelineResult, PipelineState from apache_beam.testing import pipeline_verifiers as verifiers +from apache_beam.testing.test_utils import patch_retry try: # pylint: disable=wrong-import-order, wrong-import-position diff --git a/sdks/python/apache_beam/testing/test_pipeline.py b/sdks/python/apache_beam/testing/test_pipeline.py index 83802423dbca..ea0403371e60 100644 --- a/sdks/python/apache_beam/testing/test_pipeline.py +++ b/sdks/python/apache_beam/testing/test_pipeline.py @@ -20,12 +20,12 @@ import argparse import shlex +from nose.plugins.skip import SkipTest + from apache_beam.internal import pickler +from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.pipeline import Pipeline from apache_beam.runners.runner import PipelineState -from apache_beam.options.pipeline_options import PipelineOptions -from nose.plugins.skip import SkipTest - __all__ = [ 'TestPipeline', @@ -143,7 +143,7 @@ def get_full_options_as_args(self, **extra_opts): appending, and will be unpickled later in the TestRunner. """ options = list(self.options_list) - for k, v in extra_opts.items(): + for k, v in list(extra_opts.items()): if not v: continue elif isinstance(v, bool) and v: diff --git a/sdks/python/apache_beam/testing/test_pipeline_test.py b/sdks/python/apache_beam/testing/test_pipeline_test.py index 747d64c7409f..c642c65a7b7d 100644 --- a/sdks/python/apache_beam/testing/test_pipeline_test.py +++ b/sdks/python/apache_beam/testing/test_pipeline_test.py @@ -20,12 +20,12 @@ import logging import unittest -from hamcrest.core.base_matcher import BaseMatcher from hamcrest.core.assert_that import assert_that as hc_assert_that +from hamcrest.core.base_matcher import BaseMatcher from apache_beam.internal import pickler -from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.testing.test_pipeline import TestPipeline # A simple matcher that is ued for testing extra options appending. diff --git a/sdks/python/apache_beam/testing/test_stream.py b/sdks/python/apache_beam/testing/test_stream.py index 7989fb2eee94..ef2eb10debfc 100644 --- a/sdks/python/apache_beam/testing/test_stream.py +++ b/sdks/python/apache_beam/testing/test_stream.py @@ -20,19 +20,18 @@ For internal use only; no backwards-compatibility guarantees. """ -from abc import ABCMeta -from abc import abstractmethod - -from apache_beam import coders -from apache_beam import core -from apache_beam import pvalue -from apache_beam.transforms import PTransform -from apache_beam.transforms import window +from abc import ABCMeta, abstractmethod +from builtins import object + +from future.utils import with_metaclass +from past.builtins import cmp + +from apache_beam import coders, core, pvalue +from apache_beam.transforms import PTransform, window from apache_beam.transforms.window import TimestampedValue from apache_beam.utils import timestamp from apache_beam.utils.windowed_value import WindowedValue - __all__ = [ 'Event', 'ElementEvent', @@ -42,11 +41,9 @@ ] -class Event(object): +class Event(with_metaclass(ABCMeta, object)): """Test stream event to be emitted during execution of a TestStream.""" - __metaclass__ = ABCMeta - def __cmp__(self, other): if type(self) is not type(other): return cmp(type(self), type(other)) diff --git a/sdks/python/apache_beam/testing/test_stream_test.py b/sdks/python/apache_beam/testing/test_stream_test.py index b7ca141f0598..caafe4f245d4 100644 --- a/sdks/python/apache_beam/testing/test_stream_test.py +++ b/sdks/python/apache_beam/testing/test_stream_test.py @@ -20,16 +20,13 @@ import unittest import apache_beam as beam -from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import StandardOptions +from apache_beam.options.pipeline_options import (PipelineOptions, + StandardOptions) from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.test_stream import ElementEvent -from apache_beam.testing.test_stream import ProcessingTimeEvent -from apache_beam.testing.test_stream import TestStream -from apache_beam.testing.test_stream import WatermarkEvent +from apache_beam.testing.test_stream import (ElementEvent, ProcessingTimeEvent, + TestStream, WatermarkEvent) from apache_beam.testing.util import assert_that, equal_to -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.window import TimestampedValue +from apache_beam.transforms.window import FixedWindows, TimestampedValue from apache_beam.utils import timestamp from apache_beam.utils.windowed_value import WindowedValue diff --git a/sdks/python/apache_beam/testing/test_utils.py b/sdks/python/apache_beam/testing/test_utils.py index 26ca03d1375d..de7188795b67 100644 --- a/sdks/python/apache_beam/testing/test_utils.py +++ b/sdks/python/apache_beam/testing/test_utils.py @@ -22,12 +22,12 @@ import hashlib import imp + from mock import Mock, patch from apache_beam.io.filesystems import FileSystems from apache_beam.utils import retry - DEFAULT_HASHING_ALG = 'sha1' diff --git a/sdks/python/apache_beam/testing/test_utils_test.py b/sdks/python/apache_beam/testing/test_utils_test.py index bee0bd361e27..093e1f175d25 100644 --- a/sdks/python/apache_beam/testing/test_utils_test.py +++ b/sdks/python/apache_beam/testing/test_utils_test.py @@ -20,6 +20,7 @@ import logging import tempfile import unittest + from mock import patch from apache_beam.io.filesystem import BeamIOError diff --git a/sdks/python/apache_beam/testing/util.py b/sdks/python/apache_beam/testing/util.py index 959f25f31873..6b1d92c77918 100644 --- a/sdks/python/apache_beam/testing/util.py +++ b/sdks/python/apache_beam/testing/util.py @@ -24,14 +24,11 @@ from apache_beam import pvalue from apache_beam.transforms import window -from apache_beam.transforms.core import Create -from apache_beam.transforms.core import Map -from apache_beam.transforms.core import WindowInto -from apache_beam.transforms.util import CoGroupByKey +from apache_beam.transforms.core import Create, Map, WindowInto from apache_beam.transforms.ptransform import PTransform +from apache_beam.transforms.util import CoGroupByKey from apache_beam.utils.annotations import experimental - __all__ = [ 'assert_that', 'equal_to', @@ -104,7 +101,7 @@ def expand(self, pcoll): | "ToVoidKey" >> Map(lambda v: (None, v))) _ = ((keyed_singleton, keyed_actual) | "Group" >> CoGroupByKey() - | "Unkey" >> Map(lambda (k, (_, actual_values)): actual_values) + | "Unkey" >> Map(lambda k___actual_values: k___actual_values[1][1]) | "Match" >> Map(matcher)) def default_label(self): diff --git a/sdks/python/apache_beam/transforms/combiners.py b/sdks/python/apache_beam/transforms/combiners.py index 875306f80829..d283efee6f6c 100644 --- a/sdks/python/apache_beam/transforms/combiners.py +++ b/sdks/python/apache_beam/transforms/combiners.py @@ -17,25 +17,18 @@ """A library of basic combiner PTransform subclasses.""" -from __future__ import absolute_import +from __future__ import absolute_import, division import operator import random +from builtins import object, zip -from apache_beam.transforms import core -from apache_beam.transforms import cy_combiners -from apache_beam.transforms import ptransform -from apache_beam.transforms.display import DisplayDataItem -from apache_beam.typehints import Any -from apache_beam.typehints import Dict -from apache_beam.typehints import KV -from apache_beam.typehints import List -from apache_beam.typehints import Tuple -from apache_beam.typehints import TypeVariable -from apache_beam.typehints import Union -from apache_beam.typehints import with_input_types -from apache_beam.typehints import with_output_types +from past.utils import old_div +from apache_beam.transforms import core, cy_combiners, ptransform +from apache_beam.transforms.display import DisplayDataItem +from apache_beam.typehints import (KV, Any, Dict, List, Tuple, TypeVariable, + Union, with_input_types, with_output_types) __all__ = [ 'Count', @@ -70,7 +63,7 @@ def expand(self, pcoll): # TODO(laolu): This type signature is overly restrictive. This should be # more general. -@with_input_types(Union[float, int, long]) +@with_input_types(Union[float, int, int]) @with_output_types(float) class MeanCombineFn(core.CombineFn): """CombineFn for computing an arithmetic mean.""" @@ -78,17 +71,19 @@ class MeanCombineFn(core.CombineFn): def create_accumulator(self): return (0, 0) - def add_input(self, (sum_, count), element): + def add_input(self, xxx_todo_changeme, element): + (sum_, count) = xxx_todo_changeme return sum_ + element, count + 1 def merge_accumulators(self, accumulators): - sums, counts = zip(*accumulators) + sums, counts = list(zip(*accumulators)) return sum(sums), sum(counts) - def extract_output(self, (sum_, count)): + def extract_output(self, xxx_todo_changeme1): + (sum_, count) = xxx_todo_changeme1 if count == 0: return float('NaN') - return sum_ / float(count) + return old_div(sum_, float(count)) def for_input_type(self, input_type): if input_type is int: @@ -432,7 +427,7 @@ def create_accumulator(self): def merge_accumulators(self, accumulators): return [c.merge_accumulators(a) - for c, a in zip(self._combiners, zip(*accumulators))] + for c, a in zip(self._combiners, list(zip(*accumulators)))] def extract_output(self, accumulator): return tuple([c.extract_output(a) diff --git a/sdks/python/apache_beam/transforms/combiners_test.py b/sdks/python/apache_beam/transforms/combiners_test.py index cd2b5956fef9..6f843975e51b 100644 --- a/sdks/python/apache_beam/transforms/combiners_test.py +++ b/sdks/python/apache_beam/transforms/combiners_test.py @@ -16,18 +16,19 @@ # """Unit tests for our libraries of combine PTransforms.""" +from __future__ import division import unittest +from builtins import range import hamcrest as hc +from past.utils import old_div import apache_beam as beam -from apache_beam.testing.test_pipeline import TestPipeline import apache_beam.transforms.combiners as combine +from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, equal_to -from apache_beam.transforms.core import CombineGlobally -from apache_beam.transforms.core import Create -from apache_beam.transforms.core import Map +from apache_beam.transforms.core import CombineGlobally, Create, Map from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.transforms.ptransform import PTransform @@ -43,7 +44,7 @@ def test_builtin_combines(self): pipeline = TestPipeline() vals = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6] - mean = sum(vals) / float(len(vals)) + mean = old_div(sum(vals), float(len(vals))) size = len(vals) # First for global combines. @@ -134,9 +135,10 @@ def test_combine_fn(combine_fn, shards, expected): final_accumulator = combine_fn.merge_accumulators(accumulators) self.assertEqual(combine_fn.extract_output(final_accumulator), expected) - test_combine_fn(combine.TopCombineFn(3), [range(10), range(10)], [9, 9, 8]) + test_combine_fn(combine.TopCombineFn(3), + [list(range(10)), list(range(10))], [9, 9, 8]) test_combine_fn(combine.TopCombineFn(5), - [range(1000), range(100), range(1001)], + [list(range(1000)), list(range(100)), list(range(1001))], [1000, 999, 999, 998, 998]) def test_combine_per_key_top_display_data(self): @@ -220,7 +222,7 @@ def is_good_sample(actual): with TestPipeline() as pipeline: pcoll = pipeline | 'start' >> Create([1, 1, 2, 2]) - for ix in xrange(9): + for ix in range(9): assert_that( pcoll | 'sample-%d' % ix >> combine.Sample.FixedSizeGlobally(3), is_good_sample, @@ -229,7 +231,7 @@ def is_good_sample(actual): def test_per_key_sample(self): pipeline = TestPipeline() pcoll = pipeline | 'start-perkey' >> Create( - sum(([(i, 1), (i, 1), (i, 2), (i, 2)] for i in xrange(9)), [])) + sum(([(i, 1), (i, 1), (i, 2), (i, 2)] for i in range(9)), [])) result = pcoll | 'sample' >> combine.Sample.FixedSizePerKey(3) def matcher(): @@ -250,7 +252,7 @@ def test_tuple_combine_fn(self): | Create([('a', 100, 0.0), ('b', 10, -1), ('c', 1, 100)]) | beam.CombineGlobally(combine.TupleCombineFn( max, combine.MeanCombineFn(), sum)).without_defaults()) - assert_that(result, equal_to([('c', 111.0 / 3, 99.0)])) + assert_that(result, equal_to([('c', old_div(111.0, 3), 99.0)])) def test_tuple_combine_fn_without_defaults(self): with TestPipeline() as p: @@ -260,7 +262,7 @@ def test_tuple_combine_fn_without_defaults(self): | beam.CombineGlobally( combine.TupleCombineFn(min, combine.MeanCombineFn(), max) .with_common_input()).without_defaults()) - assert_that(result, equal_to([(1, 7.0 / 4, 3)])) + assert_that(result, equal_to([(1, old_div(7.0, 4), 3)])) def test_to_list_and_to_dict(self): pipeline = TestPipeline() @@ -283,7 +285,7 @@ def match(actual): def matcher(): def match(actual): equal_to([1])([len(actual)]) - equal_to(pairs)(actual[0].iteritems()) + equal_to(pairs)(iter(actual[0].items())) return match assert_that(result, matcher()) pipeline.run() diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index d6f56d2d0cf0..d259c32a2c5e 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -17,46 +17,35 @@ """Core PTransform subclasses, such as FlatMap, GroupByKey, and Map.""" -from __future__ import absolute_import +from __future__ import absolute_import, division import copy import inspect import types +from builtins import map, next, object, range from google.protobuf import wrappers_pb2 +from past.builtins import basestring +from past.utils import old_div -from apache_beam import pvalue -from apache_beam import typehints -from apache_beam import coders +from apache_beam import coders, pvalue, typehints from apache_beam.coders import typecoders -from apache_beam.internal import pickler -from apache_beam.internal import util +from apache_beam.internal import pickler, util +from apache_beam.options.pipeline_options import TypeOptions from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.transforms import ptransform -from apache_beam.transforms.display import DisplayDataItem -from apache_beam.transforms.display import HasDisplayData -from apache_beam.transforms.ptransform import PTransform -from apache_beam.transforms.ptransform import PTransformWithSideInputs -from apache_beam.transforms.window import MIN_TIMESTAMP -from apache_beam.transforms.window import TimestampCombiner -from apache_beam.transforms.window import WindowedValue -from apache_beam.transforms.window import TimestampedValue -from apache_beam.transforms.window import GlobalWindows -from apache_beam.transforms.window import WindowFn -from apache_beam.typehints import Any -from apache_beam.typehints import Iterable -from apache_beam.typehints import KV -from apache_beam.typehints import trivial_inference -from apache_beam.typehints import Union -from apache_beam.typehints.decorators import get_type_hints -from apache_beam.typehints.decorators import TypeCheckError -from apache_beam.typehints.decorators import WithTypeHints +from apache_beam.transforms.display import DisplayDataItem, HasDisplayData +from apache_beam.transforms.ptransform import (PTransform, + PTransformWithSideInputs) +from apache_beam.transforms.window import (MIN_TIMESTAMP, GlobalWindows, + TimestampCombiner, TimestampedValue, + WindowedValue, WindowFn) +from apache_beam.typehints import KV, Any, Iterable, Union, trivial_inference +from apache_beam.typehints.decorators import (TypeCheckError, WithTypeHints, + get_type_hints) from apache_beam.typehints.trivial_inference import element_type from apache_beam.typehints.typehints import is_consistent_with -from apache_beam.utils import proto_utils -from apache_beam.utils import urns -from apache_beam.options.pipeline_options import TypeOptions - +from apache_beam.utils import proto_utils, urns __all__ = [ 'DoFn', @@ -233,10 +222,10 @@ def is_process_bounded(self): """Checks if an object is a bound method on an instance.""" if not isinstance(self.process, types.MethodType): return False # Not a method - if self.process.im_self is None: + if self.process.__self__ is None: return False # Method is not bound - if issubclass(self.process.im_class, type) or \ - self.process.im_class is types.ClassType: + if issubclass(self.process.__self__.__class__, type) or \ + self.process.__self__.__class__ is type: return False # Method is a classmethod return True @@ -249,7 +238,7 @@ def _fn_takes_side_inputs(fn): except TypeError: # We can't tell; maybe it does. return True - is_bound = isinstance(fn, types.MethodType) and fn.im_self is not None + is_bound = isinstance(fn, types.MethodType) and fn.__self__ is not None return len(argspec.args) > 1 + is_bound or argspec.varargs or argspec.keywords @@ -700,7 +689,8 @@ def with_outputs(self, *tags, **main_kw): """ main_tag = main_kw.pop('main', None) if main_kw: - raise ValueError('Unexpected keyword arguments: %s' % main_kw.keys()) + raise ValueError( + 'Unexpected keyword arguments: %s' % list(main_kw.keys())) return _MultiParDo(self, tags, main_tag) def _pardo_fn_data(self): @@ -976,7 +966,7 @@ def add_input_types(transform): KV[None, pcoll.element_type])) | 'CombinePerKey' >> CombinePerKey( self.fn, *self.args, **self.kwargs) - | 'UnKey' >> Map(lambda (k, v): v)) + | 'UnKey' >> Map(lambda k_v: k_v[1])) if not self.has_defaults and not self.as_view: return combined @@ -1518,7 +1508,7 @@ def __init__(self, **kwargs): super(Flatten, self).__init__() self.pipeline = kwargs.pop('pipeline', None) if kwargs: - raise ValueError('Unexpected keyword arguments: %s' % kwargs.keys()) + raise ValueError('Unexpected keyword arguments: %s' % list(kwargs.keys())) def _extract_input_pvalues(self, pvalueish): try: @@ -1567,7 +1557,7 @@ def __init__(self, value): raise TypeError('PTransform Create: Refusing to treat string as ' 'an iterable. (string=%r)' % value) elif isinstance(value, dict): - value = value.items() + value = list(value.items()) self.value = tuple(value) def infer_output_type(self, unused_input_type): @@ -1593,7 +1583,7 @@ def get_windowing(self, unused_inputs): @staticmethod def _create_source_from_iterable(values, coder): - return Create._create_source(map(coder.encode, values), coder) + return Create._create_source(list(map(coder.encode, values)), coder) @staticmethod def _create_source(serialized_values, coder): @@ -1639,16 +1629,17 @@ def split(self, desired_bundle_size, start_position=None, if stop_position is None: stop_position = len(self._serialized_values) - avg_size_per_value = self._total_size / len(self._serialized_values) + avg_size_per_value = old_div(self._total_size, + len(self._serialized_values)) num_values_per_split = max( - int(desired_bundle_size / avg_size_per_value), 1) + int(old_div(desired_bundle_size, avg_size_per_value)), 1) start = start_position while start < stop_position: end = min(start + num_values_per_split, stop_position) remaining = stop_position - end # Avoid having a too small bundle at the end. - if remaining < (num_values_per_split / 4): + if remaining < (old_div(num_values_per_split, 4)): end = stop_position sub_source = Create._create_source( diff --git a/sdks/python/apache_beam/transforms/create_test.py b/sdks/python/apache_beam/transforms/create_test.py index 55ad7f32982d..794248ffd57a 100644 --- a/sdks/python/apache_beam/transforms/create_test.py +++ b/sdks/python/apache_beam/transforms/create_test.py @@ -16,12 +16,16 @@ # """Unit tests for the Create and _CreateSource classes.""" +from __future__ import division + import unittest +from builtins import range -from apache_beam.io import source_test_utils +from past.utils import old_div from apache_beam import Create from apache_beam.coders import FastPrimitivesCoder +from apache_beam.io import source_test_utils from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, equal_to @@ -32,13 +36,13 @@ def setUp(self): def test_create_transform(self): with TestPipeline() as p: - assert_that(p | Create(range(10)), equal_to(range(10))) + assert_that(p | Create(list(range(10))), equal_to(list(range(10)))) def test_create_source_read(self): self.check_read([], self.coder) self.check_read([1], self.coder) # multiple values. - self.check_read(range(10), self.coder) + self.check_read(list(range(10)), self.coder) def check_read(self, values, coder): source = Create._create_source_from_iterable(values, coder) @@ -48,7 +52,7 @@ def check_read(self, values, coder): def test_create_source_read_with_initial_splits(self): self.check_read_with_initial_splits([], self.coder, num_splits=2) self.check_read_with_initial_splits([1], self.coder, num_splits=2) - values = range(8) + values = list(range(8)) # multiple values with a single split. self.check_read_with_initial_splits(values, self.coder, num_splits=1) # multiple values with a single split with a large desired bundle size @@ -69,7 +73,7 @@ def check_read_with_initial_splits(self, values, coder, num_splits): from the split sources. """ source = Create._create_source_from_iterable(values, coder) - desired_bundle_size = source._total_size / num_splits + desired_bundle_size = old_div(source._total_size, num_splits) splits = source.split(desired_bundle_size) splits_info = [ (split.source, split.start_position, split.stop_position) @@ -78,11 +82,11 @@ def check_read_with_initial_splits(self, values, coder, num_splits): (source, None, None), splits_info) def test_create_source_read_reentrant(self): - source = Create._create_source_from_iterable(range(9), self.coder) + source = Create._create_source_from_iterable(list(range(9)), self.coder) source_test_utils.assert_reentrant_reads_succeed((source, None, None)) def test_create_source_read_reentrant_with_initial_splits(self): - source = Create._create_source_from_iterable(range(24), self.coder) + source = Create._create_source_from_iterable(list(range(24)), self.coder) for split in source.split(desired_bundle_size=5): source_test_utils.assert_reentrant_reads_succeed((split.source, split.start_position, @@ -90,10 +94,10 @@ def test_create_source_read_reentrant_with_initial_splits(self): def test_create_source_dynamic_splitting(self): # 2 values - source = Create._create_source_from_iterable(range(2), self.coder) + source = Create._create_source_from_iterable(list(range(2)), self.coder) source_test_utils.assert_split_at_fraction_exhaustive(source) # Multiple values. - source = Create._create_source_from_iterable(range(11), self.coder) + source = Create._create_source_from_iterable(list(range(11)), self.coder) source_test_utils.assert_split_at_fraction_exhaustive( source, perform_multi_threaded_test=True) @@ -111,7 +115,7 @@ def test_create_source_progress(self): split_points_report.append(range_tracker.split_points()) self.assertEqual( - [float(i) / num_values for i in range(num_values)], + [old_div(float(i), num_values) for i in range(num_values)], fraction_consumed_report) expected_split_points_report = [ diff --git a/sdks/python/apache_beam/transforms/cy_combiners.py b/sdks/python/apache_beam/transforms/cy_combiners.py index 84aee212790c..21fd72d194f5 100644 --- a/sdks/python/apache_beam/transforms/cy_combiners.py +++ b/sdks/python/apache_beam/transforms/cy_combiners.py @@ -20,7 +20,11 @@ For internal use only; no backwards-compatibility guarantees. """ -from __future__ import absolute_import +from __future__ import absolute_import, division + +from builtins import object + +from past.utils import old_div from apache_beam.transforms import core @@ -156,7 +160,7 @@ def extract_output(self): self.sum %= 2**64 if self.sum >= INT64_MAX: self.sum -= 2**64 - return self.sum / self.count if self.count else _NAN + return old_div(self.sum, self.count) if self.count else _NAN class CountCombineFn(AccumulatorCombineFn): @@ -252,7 +256,7 @@ def merge(self, accumulators): self.count += accumulator.count def extract_output(self): - return self.sum / self.count if self.count else _NAN + return old_div(self.sum, self.count) if self.count else _NAN class SumFloatFn(AccumulatorCombineFn): diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 88a1feef75e1..81679d9eb674 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -39,10 +39,14 @@ from __future__ import absolute_import import calendar -from datetime import datetime, timedelta import inspect import json +import sys +from builtins import object +from datetime import datetime, timedelta +if sys.version_info[0] >= 3: + unicode = str __all__ = ['HasDisplayData', 'DisplayDataItem', 'DisplayData'] @@ -92,7 +96,7 @@ def __init__(self, namespace, display_data_dict): def _populate_items(self, display_data_dict): """ Populates the list of display data items. """ - for key, element in display_data_dict.items(): + for key, element in list(display_data_dict.items()): if isinstance(element, HasDisplayData): subcomponent_display_data = DisplayData(element._namespace(), element.display_data()) @@ -139,7 +143,7 @@ def create_from_options(cls, pipeline_options): items = {k: (v if DisplayDataItem._get_value_type(v) is not None else str(v)) - for k, v in pipeline_options.display_data().items()} + for k, v in list(pipeline_options.display_data().items())} return cls(pipeline_options._namespace(), items) @classmethod diff --git a/sdks/python/apache_beam/transforms/display_test.py b/sdks/python/apache_beam/transforms/display_test.py index 15f178645bc3..d5578a6a29b0 100644 --- a/sdks/python/apache_beam/transforms/display_test.py +++ b/sdks/python/apache_beam/transforms/display_test.py @@ -19,17 +19,16 @@ from __future__ import absolute_import -from datetime import datetime import unittest +from datetime import datetime import hamcrest as hc from hamcrest.core.base_matcher import BaseMatcher import apache_beam as beam -from apache_beam.transforms.display import HasDisplayData -from apache_beam.transforms.display import DisplayData -from apache_beam.transforms.display import DisplayDataItem from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.transforms.display import (DisplayData, DisplayDataItem, + HasDisplayData) class DisplayDataItemMatcher(BaseMatcher): @@ -161,13 +160,13 @@ def test_create_list_display_data(self): def test_unicode_type_display_data(self): class MyDoFn(beam.DoFn): def display_data(self): - return {'unicode_string': unicode('my string'), + return {'unicode_string': str('my string'), 'unicode_literal_string': u'my literal string'} fn = MyDoFn() dd = DisplayData.create_from(fn) for item in dd.items: - self.assertEqual(item.type, 'STRING') + self.assertEqual(item.type, 'STRING', repr(item) + "should be string") def test_base_cases(self): """ Tests basic display data cases (key:value, key:dict) diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index f6e08ca9c0ce..94528c307932 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -41,24 +41,20 @@ class and wrapper class that allows lambda functions to be used as import operator import os import sys +from builtins import hex, object, zip +from functools import reduce from google.protobuf import wrappers_pb2 -from apache_beam import error -from apache_beam import pvalue -from apache_beam.internal import pickler -from apache_beam.internal import util -from apache_beam.transforms.display import HasDisplayData -from apache_beam.transforms.display import DisplayDataItem +from apache_beam import error, pvalue +from apache_beam.internal import pickler, util +from apache_beam.transforms.display import DisplayDataItem, HasDisplayData from apache_beam.typehints import typehints -from apache_beam.typehints.decorators import getcallargs_forhints -from apache_beam.typehints.decorators import TypeCheckError -from apache_beam.typehints.decorators import WithTypeHints +from apache_beam.typehints.decorators import (TypeCheckError, WithTypeHints, + getcallargs_forhints) from apache_beam.typehints.trivial_inference import instance_to_type from apache_beam.typehints.typehints import validate_composite_type_param -from apache_beam.utils import proto_utils -from apache_beam.utils import urns - +from apache_beam.utils import proto_utils, urns __all__ = [ 'PTransform', @@ -171,10 +167,10 @@ def visit_tuple(self, pvalueish, sibling, pairs, context): def visit_dict(self, pvalueish, sibling, pairs, context): if isinstance(sibling, dict): - for key, p in pvalueish.items(): + for key, p in list(pvalueish.items()): self.visit(p, sibling.get(key), pairs, key) else: - for p in pvalueish.values(): + for p in list(pvalueish.values()): self.visit(p, sibling, pairs, context) @@ -422,7 +418,7 @@ def _dict_tuple_leaves(pvalueish): for p in _dict_tuple_leaves(a): yield p elif isinstance(pvalueish, dict): - for a in pvalueish.values(): + for a in list(pvalueish.values()): for p in _dict_tuple_leaves(a): yield p else: @@ -518,7 +514,7 @@ def __init__(self, fn, *args, **kwargs): super(PTransformWithSideInputs, self).__init__() if (any([isinstance(v, pvalue.PCollection) for v in args]) or - any([isinstance(v, pvalue.PCollection) for v in kwargs.itervalues()])): + any([isinstance(v, pvalue.PCollection) for v in kwargs.values()])): raise error.SideInputError( 'PCollection used directly as side input argument. Specify ' 'AsIter(pcollection) or AsSingleton(pcollection) to indicate how the ' @@ -571,7 +567,7 @@ def with_input_types( for si in side_inputs_arg_hints: validate_composite_type_param(si, 'Type hints for a PTransform') - for si in side_input_kwarg_hints.values(): + for si in list(side_input_kwarg_hints.values()): validate_composite_type_param(si, 'Type hints for a PTransform') self.side_inputs_types = side_inputs_arg_hints @@ -589,11 +585,11 @@ def element_type(side_input): return instance_to_type(side_input) arg_types = [pvalueish.element_type] + [element_type(v) for v in args] - kwargs_types = {k: element_type(v) for (k, v) in kwargs.items()} + kwargs_types = {k: element_type(v) for (k, v) in list(kwargs.items())} argspec_fn = self._process_argspec_fn() bindings = getcallargs_forhints(argspec_fn, *arg_types, **kwargs_types) hints = getcallargs_forhints(argspec_fn, *type_hints[0], **type_hints[1]) - for arg, hint in hints.items(): + for arg, hint in list(hints.items()): if arg.startswith('%unknown%'): continue if hint is None: @@ -711,8 +707,8 @@ def label_from_callable(fn): elif hasattr(fn, '__name__'): if fn.__name__ == '': return '' % ( - os.path.basename(fn.func_code.co_filename), - fn.func_code.co_firstlineno) + os.path.basename(fn.__code__.co_filename), + fn.__code__.co_firstlineno) return fn.__name__ return str(fn) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index efc597877b29..2ddf0dbbd48e 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -17,35 +17,37 @@ """Unit tests for the PTransform and descendants.""" -from __future__ import absolute_import +from __future__ import absolute_import, division, print_function import operator import re import unittest +from builtins import map, range, zip +from functools import reduce import hamcrest as hc from nose.plugins.attrib import attr +from past.utils import old_div import apache_beam as beam +import apache_beam.pvalue as pvalue +import apache_beam.transforms.combiners as combine +import apache_beam.typehints as typehints +from apache_beam.io.iobase import Read from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter -from apache_beam.io.iobase import Read from apache_beam.options.pipeline_options import TypeOptions -import apache_beam.pvalue as pvalue from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, equal_to from apache_beam.transforms import window from apache_beam.transforms.core import _GroupByKeyOnly -import apache_beam.transforms.combiners as combine from apache_beam.transforms.display import DisplayData, DisplayDataItem from apache_beam.transforms.ptransform import PTransform -import apache_beam.typehints as typehints -from apache_beam.typehints import with_input_types -from apache_beam.typehints import with_output_types +from apache_beam.typehints import with_input_types, with_output_types from apache_beam.typehints.typehints_test import TypeHintTestCase +from apache_beam.utils.test_utils import _rewrite_typehint_string from apache_beam.utils.windowed_value import WindowedValue - # Disable frequent lint warning due to pipe operator for chaining transforms. # pylint: disable=expression-not-assigned @@ -351,24 +353,26 @@ class _MeanCombineFn(beam.CombineFn): def create_accumulator(self): return (0, 0) - def add_input(self, (sum_, count), element): + def add_input(self, xxx_todo_changeme, element): + (sum_, count) = xxx_todo_changeme return sum_ + element, count + 1 def merge_accumulators(self, accumulators): - sums, counts = zip(*accumulators) + sums, counts = list(zip(*accumulators)) return sum(sums), sum(counts) - def extract_output(self, (sum_, count)): + def extract_output(self, xxx_todo_changeme3): + (sum_, count) = xxx_todo_changeme3 if not count: return float('nan') - return sum_ / float(count) + return old_div(sum_, float(count)) def test_combine_with_combine_fn(self): vals = [1, 2, 3, 4, 5, 6, 7] pipeline = TestPipeline() pcoll = pipeline | 'Start' >> beam.Create(vals) result = pcoll | 'Mean' >> beam.CombineGlobally(self._MeanCombineFn()) - assert_that(result, equal_to([sum(vals) / len(vals)])) + assert_that(result, equal_to([old_div(sum(vals), len(vals))])) pipeline.run() def test_combine_with_callable(self): @@ -399,8 +403,8 @@ def test_combine_per_key_with_combine_fn(self): pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] + [('b', x) for x in vals_2])) result = pcoll | 'Mean' >> beam.CombinePerKey(self._MeanCombineFn()) - assert_that(result, equal_to([('a', sum(vals_1) / len(vals_1)), - ('b', sum(vals_2) / len(vals_2))])) + assert_that(result, equal_to([('a', old_div(sum(vals_1), len(vals_1))), + ('b', old_div(sum(vals_2), len(vals_2)))])) pipeline.run() def test_combine_per_key_with_callable(self): @@ -617,7 +621,7 @@ def test_chained_ptransforms(self): pipeline = TestPipeline() t = (beam.Map(lambda x: (x, 1)) | beam.GroupByKey() - | beam.Map(lambda (x, ones): (x, sum(ones)))) + | beam.Map(lambda x_ones: (x_ones[0], sum(x_ones[1])))) result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t assert_that(result, equal_to([('a', 2), ('b', 1)])) pipeline.run() @@ -641,7 +645,7 @@ def expand(self, pcollections): | beam.Flatten() | beam.Map(lambda x: (x, None)) | beam.GroupByKey() - | beam.Map(lambda (x, _): x)) + | beam.Map(lambda x__: x__[0])) self.assertEqual([1, 2, 3], sorted(([1, 2], [2, 3]) | DisjointUnion())) def test_apply_to_crazy_pvaluish(self): @@ -694,7 +698,7 @@ def test_chained_ptransforms(self): pipeline = TestPipeline() map1 = 'Map1' >> beam.Map(lambda x: (x, 1)) gbk = 'Gbk' >> beam.GroupByKey() - map2 = 'Map2' >> beam.Map(lambda (x, ones): (x, sum(ones))) + map2 = 'Map2' >> beam.Map(lambda x_ones2: (x_ones2[0], sum(x_ones2[1]))) t = (map1 | gbk | map2) result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t self.assertTrue('Map1|Gbk|Map2/Map1' in pipeline.applied_labels) @@ -881,7 +885,7 @@ def process(self, element, prefix): self.assertEqual("Type hint violation for 'Upper': " "requires but got for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_do_fn_pipeline_runtime_type_check_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True @@ -916,7 +920,7 @@ def process(self, element, num): self.assertEqual("Type hint violation for 'Add': " "requires but got for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_pardo_does_not_type_check_using_type_hint_decorators(self): @with_input_types(a=int) @@ -933,7 +937,7 @@ def int_to_str(a): self.assertEqual("Type hint violation for 'ToStr': " "requires but got for a", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_pardo_properly_type_checks_using_type_hint_decorators(self): @with_input_types(a=str) @@ -965,7 +969,7 @@ def test_pardo_does_not_type_check_using_type_hint_methods(self): self.assertEqual("Type hint violation for 'Upper': " "requires but got for x", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_pardo_properly_type_checks_using_type_hint_methods(self): # Pipeline should be created successfully without an error @@ -990,7 +994,7 @@ def test_map_does_not_type_check_using_type_hints_methods(self): self.assertEqual("Type hint violation for 'Upper': " "requires but got for x", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_map_properly_type_checks_using_type_hints_methods(self): # No error should be raised if this type-checks properly. @@ -1016,7 +1020,7 @@ def upper(s): self.assertEqual("Type hint violation for 'Upper': " "requires but got for s", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_map_properly_type_checks_using_type_hints_decorator(self): @with_input_types(a=bool) @@ -1043,7 +1047,7 @@ def test_filter_does_not_type_check_using_type_hints_method(self): self.assertEqual("Type hint violation for 'Below 3': " "requires but got for x", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_filter_type_checks_using_type_hints_method(self): # No error should be raised if this type-checks properly. @@ -1068,7 +1072,7 @@ def more_than_half(a): self.assertEqual("Type hint violation for 'Half': " "requires but got for a", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_filter_type_checks_using_type_hints_decorator(self): @with_input_types(b=int) @@ -1078,7 +1082,7 @@ def half(b): # Filter should deduce that it returns the same type that it takes. (self.p - | 'Str' >> beam.Create(range(5)).with_output_types(int) + | 'Str' >> beam.Create(list(range(5))).with_output_types(int) | 'Half' >> beam.Filter(half) | 'ToBool' >> beam.Map(lambda x: bool(x)) .with_input_types(int).with_output_types(bool)) @@ -1097,7 +1101,7 @@ def test_group_by_key_only_output_type_deduction(self): def test_group_by_key_output_type_deduction(self): d = (self.p - | 'Str' >> beam.Create(range(20)).with_output_types(int) + | 'Str' >> beam.Create(list(range(20))).with_output_types(int) | ('PairNegative' >> beam.Map(lambda x: (x % 5, -x)) .with_output_types(typehints.KV[int, int])) | beam.GroupByKey()) @@ -1117,7 +1121,7 @@ def test_group_by_key_only_does_not_type_check(self): self.assertEqual("Input type hint violation at F: " "expected Tuple[TypeVariable[K], TypeVariable[V]], " "got ", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_group_by_does_not_type_check(self): # Create is returning a List[int, str], rather than a KV[int, str] that is @@ -1131,7 +1135,7 @@ def test_group_by_does_not_type_check(self): self.assertEqual("Input type hint violation at T: " "expected Tuple[TypeVariable[K], TypeVariable[V]], " "got Iterable[int]", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_pipeline_checking_pardo_insufficient_type_information(self): self.p._options.view_as(TypeOptions).type_check_strictness = 'ALL_REQUIRED' @@ -1140,7 +1144,7 @@ def test_pipeline_checking_pardo_insufficient_type_information(self): # information to the ParDo. with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | 'Nums' >> beam.Create(range(5)) + | 'Nums' >> beam.Create(list(range(5))) | 'ModDup' >> beam.FlatMap(lambda x: (x % 2, x))) self.assertEqual('Pipeline type checking is enabled, however no output ' @@ -1153,7 +1157,7 @@ def test_pipeline_checking_gbk_insufficient_type_information(self): # information to GBK-only. with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | 'Nums' >> beam.Create(range(5)).with_output_types(int) + | 'Nums' >> beam.Create(list(range(5))).with_output_types(int) | 'ModDup' >> beam.Map(lambda x: (x % 2, x)) | _GroupByKeyOnly()) @@ -1190,7 +1194,7 @@ def int_to_string(x): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(ToStr): " "Type-hint for argument: 'x' violated. " "Expected an instance of , " @@ -1230,7 +1234,7 @@ def is_even_as_key(a): return (a % 2, a) (self.p - | 'Nums' >> beam.Create(range(5)).with_output_types(int) + | 'Nums' >> beam.Create(list(range(5))).with_output_types(int) | 'IsEven' >> beam.Map(is_even_as_key) | 'Parity' >> beam.GroupByKey()) @@ -1241,7 +1245,7 @@ def is_even_as_key(a): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(IsEven): " "Tuple[bool, int] hint type-constraint violated. " "The type of element #0 in the passed tuple is incorrect. " @@ -1259,7 +1263,7 @@ def is_even_as_key(a): return (a % 2 == 0, a) result = (self.p - | 'Nums' >> beam.Create(range(5)).with_output_types(int) + | 'Nums' >> beam.Create(list(range(5))).with_output_types(int) | 'IsEven' >> beam.Map(is_even_as_key) | 'Parity' >> beam.GroupByKey()) @@ -1281,7 +1285,7 @@ def test_pipeline_runtime_checking_violation_simple_type_input(self): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(ToInt): " "Type-hint for argument: 'x' violated. " "Expected an instance of , " @@ -1294,17 +1298,16 @@ def test_pipeline_runtime_checking_violation_composite_type_input(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p | beam.Create([(1, 3.0), (2, 4.9), (3, 9.5)]) - | ('Add' >> beam.FlatMap(lambda (x, y): [x + y]) + | ('Add' >> beam.FlatMap(lambda x_y: [x_y[0] + x_y[1]]) .with_input_types(typehints.Tuple[int, int]).with_output_types(int)) ) self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(Add): " - "Type-hint for argument: 'y' violated. " - "Expected an instance of , " - "instead found 3.0, an instance of .") + "Type-hint for argument: 'x_y' violated: " + "Tuple[int, int] hint type-constraint violated.") def test_pipeline_runtime_checking_violation_simple_type_output(self): self.p._options.view_as(TypeOptions).runtime_type_check = True @@ -1313,9 +1316,9 @@ def test_pipeline_runtime_checking_violation_simple_type_output(self): # The type-hinted applied via the 'returns()' method indicates the ParDo # should output an instance of type 'int', however a 'float' will be # generated instead. - print "HINTS", ('ToInt' >> beam.FlatMap( + print("HINTS", ('ToInt' >> beam.FlatMap( lambda x: [float(x)]).with_input_types(int).with_output_types( - int)).get_type_hints() + int)).get_type_hints()) with self.assertRaises(typehints.TypeCheckError) as e: (self.p | beam.Create([1, 2, 3]) @@ -1325,7 +1328,7 @@ def test_pipeline_runtime_checking_violation_simple_type_output(self): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within " "ParDo(ToInt): " "According to type-hint expected output should be " @@ -1342,14 +1345,14 @@ def test_pipeline_runtime_checking_violation_composite_type_output(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p | beam.Create([(1, 3.0), (2, 4.9), (3, 9.5)]) - | ('Swap' >> beam.FlatMap(lambda (x, y): [x + y]) + | ('Swap' >> beam.FlatMap(lambda x_y1: [x_y1[0] + x_y1[1]]) .with_input_types(typehints.Tuple[int, float]) .with_output_types(typehints.Tuple[float, int])) ) self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within " "ParDo(Swap): Tuple type constraint violated. " "Valid object instance must be of type 'tuple'. Instead, " @@ -1369,7 +1372,7 @@ def add(a, b): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(Add 1): " "Type-hint for argument: 'b' violated. " "Expected an instance of , " @@ -1388,7 +1391,7 @@ def test_pipline_runtime_checking_violation_with_side_inputs_via_method(self): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within ParDo(Add 1): " "Type-hint for argument: 'one' violated. " "Expected an instance of , " @@ -1404,8 +1407,8 @@ def sum_ints(ints): | 'T' >> beam.Create([1, 2, 3]).with_output_types(int) | 'Sum' >> beam.CombineGlobally(sum_ints)) - self.assertEqual(int, d.element_type) assert_that(d, equal_to([6])) + self.assertEqual(int, d.element_type) self.p.run() def test_combine_func_type_hint_does_not_take_iterable_using_decorator(self): @@ -1423,7 +1426,7 @@ def bad_combine(a): "All functions for a Combine PTransform must accept a " "single argument compatible with: Iterable[Any]. " "Instead a function with input type: was received.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_combine_pipeline_type_propagation_using_decorators(self): @with_output_types(int) @@ -1477,7 +1480,7 @@ def iter_mul(ints): self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within " "ParDo(Mul/CombinePerKey/LiftedCombinePerKey/ParDo(FinishCombine)): " "Tuple[TypeVariable[K], int] hint type-constraint violated. " @@ -1503,7 +1506,7 @@ def test_combine_runtime_type_check_using_methods(self): self.p._options.view_as(TypeOptions).runtime_type_check = True d = (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | ('Sum' >> beam.CombineGlobally(lambda s: sum(s)) .with_input_types(int).with_output_types(int))) @@ -1513,13 +1516,13 @@ def test_combine_runtime_type_check_using_methods(self): def test_combine_pipeline_type_check_violation_using_methods(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(range(3)).with_output_types(int) + | beam.Create(list(range(3))).with_output_types(int) | ('SortJoin' >> beam.CombineGlobally(lambda s: ''.join(sorted(s))) .with_input_types(str).with_output_types(str))) self.assertEqual("Input type hint violation at SortJoin: " "expected , got ", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_combine_runtime_type_check_violation_using_methods(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False @@ -1527,13 +1530,13 @@ def test_combine_runtime_type_check_violation_using_methods(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(range(3)).with_output_types(int) + | beam.Create(list(range(3))).with_output_types(int) | ('SortJoin' >> beam.CombineGlobally(lambda s: ''.join(sorted(s))) .with_input_types(str).with_output_types(str))) self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within " "ParDo(SortJoin/KeyWithVoid): " "Type-hint for argument: 'v' violated. " @@ -1545,7 +1548,7 @@ def test_combine_insufficient_type_hint_information(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | 'E' >> beam.Create(range(3)).with_output_types(int) + | 'E' >> beam.Create(list(range(3))).with_output_types(int) | 'SortJoin' >> beam.CombineGlobally(lambda s: ''.join(sorted(s))) | 'F' >> beam.Map(lambda x: x + 1)) @@ -1558,10 +1561,10 @@ def test_combine_insufficient_type_hint_information(self): def test_mean_globally_pipeline_checking_satisfied(self): d = (self.p - | 'C' >> beam.Create(range(5)).with_output_types(int) + | 'C' >> beam.Create(list(range(5))).with_output_types(int) | 'Mean' >> combine.Mean.Globally()) - self.assertTrue(d.element_type is float) + self.assertEqual(d.element_type, float) assert_that(d, equal_to([2.0])) self.p.run() @@ -1573,18 +1576,18 @@ def test_mean_globally_pipeline_checking_violated(self): self.assertEqual( "Type hint violation for 'ParDo(PartialGroupByKeyCombiningValues)': " - "requires Tuple[TypeVariable[K], Union[float, int, long]] " + "requires Tuple[TypeVariable[K], Union[float, int]] " "but got Tuple[None, str] for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_mean_globally_runtime_checking_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True d = (self.p - | 'C' >> beam.Create(range(5)).with_output_types(int) + | 'C' >> beam.Create(list(range(5))).with_output_types(int) | 'Mean' >> combine.Mean.Globally()) - self.assertTrue(d.element_type is float) + self.assertEqual(d.element_type, float) assert_that(d, equal_to([2.0])) self.p.run() @@ -1606,11 +1609,11 @@ def test_mean_globally_runtime_checking_violated(self): "the passed Iterable is incorrect: Union[int, float] " "type-constraint violated. Expected an instance of one " "of: ('int', 'float'), received str instead.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_mean_per_key_pipeline_checking_satisfied(self): d = (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | ('EvenGroup' >> beam.Map(lambda x: (not x % 2, x)) .with_output_types(typehints.KV[bool, int])) | 'EvenMean' >> combine.Mean.PerKey()) @@ -1622,7 +1625,7 @@ def test_mean_per_key_pipeline_checking_satisfied(self): def test_mean_per_key_pipeline_checking_violated(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(map(str, range(5))).with_output_types(str) + | beam.Create(list(map(str, list(range(5))))).with_output_types(str) | ('UpperPair' >> beam.Map(lambda x: (x.upper(), x)) .with_output_types(typehints.KV[str, str])) | 'EvenMean' >> combine.Mean.PerKey()) @@ -1630,15 +1633,15 @@ def test_mean_per_key_pipeline_checking_violated(self): self.assertEqual( "Type hint violation for 'ParDo(PartialGroupByKeyCombiningValues)': " - "requires Tuple[TypeVariable[K], Union[float, int, long]] " - "but got Tuple[str, str] for element", - e.exception.message) + "requires Tuple[TypeVariable[K], Union[float, int]] but got " + "Tuple[str, str] for element", + _rewrite_typehint_string(e.exception.message)) def test_mean_per_key_runtime_checking_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True d = (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | ('OddGroup' >> beam.Map(lambda x: (bool(x % 2), x)) .with_output_types(typehints.KV[bool, int])) | 'OddMean' >> combine.Mean.PerKey()) @@ -1653,31 +1656,26 @@ def test_mean_per_key_runtime_checking_violated(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | ('OddGroup' >> beam.Map(lambda x: (x, str(bool(x % 2)))) .with_output_types(typehints.KV[int, str])) | 'OddMean' >> combine.Mean.PerKey()) self.p.run() self.assertStartswith( - e.exception.message, + _rewrite_typehint_string(e.exception.message), "Runtime type violation detected within " "ParDo(OddMean/CombinePerKey(MeanCombineFn)/LiftedCombinePerKey/" "ParDo(PartialGroupByKeyCombiningValues)): " "Type-hint for argument: 'element' violated: " - "Tuple[TypeVariable[K], Union[float, int, long]]" - " hint type-constraint violated. " - "The type of element #1 in the passed tuple is incorrect. " - "Union[float, int, long] type-constraint violated. " - "Expected an instance of one of: ('float', 'int', 'long'), " - "received str instead.") + "Tuple[TypeVariable[K], Union[float, int") def test_count_globally_pipeline_type_checking_satisfied(self): d = (self.p - | 'P' >> beam.Create(range(5)).with_output_types(int) + | 'P' >> beam.Create(list(range(5))).with_output_types(int) | 'CountInt' >> combine.Count.Globally()) - self.assertTrue(d.element_type is int) + self.assertEqual(d.element_type, int) assert_that(d, equal_to([5])) self.p.run() @@ -1685,16 +1683,16 @@ def test_count_globally_runtime_type_checking_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True d = (self.p - | 'P' >> beam.Create(range(5)).with_output_types(int) + | 'P' >> beam.Create(list(range(5))).with_output_types(int) | 'CountInt' >> combine.Count.Globally()) - self.assertTrue(d.element_type is int) + self.assertEqual(d.element_type, int) assert_that(d, equal_to([5])) self.p.run() def test_count_perkey_pipeline_type_checking_satisfied(self): d = (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | ('EvenGroup' >> beam.Map(lambda x: (not x % 2, x)) .with_output_types(typehints.KV[bool, int])) | 'CountInt' >> combine.Count.PerKey()) @@ -1706,14 +1704,14 @@ def test_count_perkey_pipeline_type_checking_satisfied(self): def test_count_perkey_pipeline_type_checking_violated(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(range(5)).with_output_types(int) + | beam.Create(list(range(5))).with_output_types(int) | 'CountInt' >> combine.Count.PerKey()) self.assertEqual( "Type hint violation for 'ParDo(PartialGroupByKeyCombiningValues)': " "requires Tuple[TypeVariable[K], Any] " "but got for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_count_perkey_runtime_type_checking_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True @@ -1764,7 +1762,7 @@ def test_count_perelement_runtime_type_checking_satisfied(self): def test_top_of_pipeline_checking_satisfied(self): d = (self.p - | beam.Create(range(5, 11)).with_output_types(int) + | beam.Create(list(range(5, 11))).with_output_types(int) | 'Top 3' >> combine.Top.Of(3, lambda x, y: x < y)) self.assertCompatible(typehints.Iterable[int], @@ -1786,7 +1784,7 @@ def test_top_of_runtime_checking_satisfied(self): def test_per_key_pipeline_checking_violated(self): with self.assertRaises(typehints.TypeCheckError) as e: (self.p - | beam.Create(range(100)).with_output_types(int) + | beam.Create(list(range(100))).with_output_types(int) | 'Num + 1' >> beam.Map(lambda x: x + 1).with_output_types(int) | 'TopMod' >> combine.Top.PerKey(1, lambda a, b: a < b)) @@ -1794,11 +1792,11 @@ def test_per_key_pipeline_checking_violated(self): "Type hint violation for 'ParDo(PartialGroupByKeyCombiningValues)': " "requires Tuple[TypeVariable[K], TypeVariable[T]] " "but got for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_per_key_pipeline_checking_satisfied(self): d = (self.p - | beam.Create(range(100)).with_output_types(int) + | beam.Create(list(range(100))).with_output_types(int) | ('GroupMod 3' >> beam.Map(lambda x: (x % 3, x)) .with_output_types(typehints.KV[int, int])) | 'TopMod' >> combine.Top.PerKey(1, lambda a, b: a < b)) @@ -1812,7 +1810,7 @@ def test_per_key_runtime_checking_satisfied(self): self.p._options.view_as(TypeOptions).runtime_type_check = True d = (self.p - | beam.Create(range(21)) + | beam.Create(list(range(21))) | ('GroupMod 3' >> beam.Map(lambda x: (x % 3, x)) .with_output_types(typehints.KV[int, int])) | 'TopMod' >> combine.Top.PerKey(1, lambda a, b: a < b)) @@ -1929,7 +1927,7 @@ def test_to_dict_pipeline_check_violated(self): "requires " "Tuple[TypeVariable[K], Tuple[TypeVariable[K], TypeVariable[V]]] " "but got Tuple[None, int] for element", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_to_dict_pipeline_check_satisfied(self): d = (self.p @@ -1966,7 +1964,7 @@ def test_runtime_type_check_python_type_error(self): # Instead the above pipeline should have triggered a regular Python runtime # TypeError. self.assertEqual("object of type 'int' has no len() [while running 'Len']", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) self.assertFalse(isinstance(e, typehints.TypeCheckError)) def test_pardo_type_inference(self): @@ -1999,7 +1997,7 @@ def test_inferred_bad_kv_type(self): self.assertEqual('Input type hint violation at GroupByKey: ' 'expected Tuple[TypeVariable[K], TypeVariable[V]], ' 'got Tuple[str, int, float]', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_inference_command_line_flag_toggle(self): self.p._options.view_as(TypeOptions).pipeline_type_check = False diff --git a/sdks/python/apache_beam/transforms/sideinputs.py b/sdks/python/apache_beam/transforms/sideinputs.py index f10cb92ed5e3..21fc919b72d1 100644 --- a/sdks/python/apache_beam/transforms/sideinputs.py +++ b/sdks/python/apache_beam/transforms/sideinputs.py @@ -26,6 +26,8 @@ from __future__ import absolute_import +from builtins import object + from apache_beam.transforms import window diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py index 650068141852..a4fd53775617 100644 --- a/sdks/python/apache_beam/transforms/sideinputs_test.py +++ b/sdks/python/apache_beam/transforms/sideinputs_test.py @@ -54,7 +54,7 @@ def run_windowed_side_inputs(self, elements, main_window_fn, side |= beam.Map(lambda x: ('k%s' % x, 'v%s' % x)) res = main | beam.Map(lambda x, s: (x, s), side_input_type(side, **kw)) if side_input_type in (beam.pvalue.AsIter, beam.pvalue.AsList): - res |= beam.Map(lambda (x, s): (x, sorted(s))) + res |= beam.Map(lambda x_s: (x_s[0], sorted(x_s[1]))) assert_that(res, equal_to(expected)) def test_global_global_windows(self): @@ -193,7 +193,7 @@ def match(actual): [[actual_elem, actual_list, actual_dict]] = actual equal_to([expected_elem])([actual_elem]) equal_to(expected_list)(actual_list) - equal_to(expected_pairs)(actual_dict.iteritems()) + equal_to(expected_pairs)(iter(actual_dict.items())) return match assert_that(results, matcher(1, a_list, some_pairs)) @@ -283,8 +283,8 @@ def matcher(expected_elem, expected_kvs): def match(actual): [[actual_elem, actual_dict1, actual_dict2]] = actual equal_to([expected_elem])([actual_elem]) - equal_to(expected_kvs)(actual_dict1.iteritems()) - equal_to(expected_kvs)(actual_dict2.iteritems()) + equal_to(expected_kvs)(iter(actual_dict1.items())) + equal_to(expected_kvs)(iter(actual_dict2.items())) return match assert_that(results, matcher(1, some_kvs)) diff --git a/sdks/python/apache_beam/transforms/timeutil.py b/sdks/python/apache_beam/transforms/timeutil.py index c0f91981d7d6..5ba47d172735 100644 --- a/sdks/python/apache_beam/transforms/timeutil.py +++ b/sdks/python/apache_beam/transforms/timeutil.py @@ -19,9 +19,10 @@ from __future__ import absolute_import -from abc import ABCMeta -from abc import abstractmethod +from abc import ABCMeta, abstractmethod +from builtins import object +from future.utils import with_metaclass __all__ = [ 'TimeDomain', @@ -44,11 +45,9 @@ def from_string(domain): raise ValueError('Unknown time domain: %s' % domain) -class TimestampCombinerImpl(object): +class TimestampCombinerImpl(with_metaclass(ABCMeta, object)): """Implementation of TimestampCombiner.""" - __metaclass__ = ABCMeta - @abstractmethod def assign_output_time(self, window, input_timestamp): pass @@ -73,11 +72,9 @@ def merge(self, unused_result_window, merging_timestamps): return self.combine_all(merging_timestamps) -class DependsOnlyOnWindow(TimestampCombinerImpl): +class DependsOnlyOnWindow(with_metaclass(ABCMeta, TimestampCombinerImpl)): """TimestampCombinerImpl that only depends on the window.""" - __metaclass__ = ABCMeta - def combine(self, output_timestamp, other_output_timestamp): return output_timestamp diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index c1fbfc54a3b8..8ed5fe5b7439 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -20,24 +20,22 @@ Triggers control when in processing time windows get emitted. """ -from abc import ABCMeta -from abc import abstractmethod import collections import copy import itertools +from abc import ABCMeta, abstractmethod +from builtins import object + +from future.utils import with_metaclass from apache_beam.coders import observable -from apache_beam.transforms import combiners -from apache_beam.transforms import core -from apache_beam.transforms.timeutil import TimeDomain -from apache_beam.transforms.window import GlobalWindow -from apache_beam.transforms.window import TimestampCombiner -from apache_beam.transforms.window import WindowedValue -from apache_beam.transforms.window import WindowFn from apache_beam.portability.api import beam_runner_api_pb2 -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.utils.timestamp import TIME_GRANULARITY +from apache_beam.transforms import combiners, core +from apache_beam.transforms.timeutil import TimeDomain +from apache_beam.transforms.window import (GlobalWindow, TimestampCombiner, + WindowedValue, WindowFn) +from apache_beam.utils.timestamp import (MAX_TIMESTAMP, MIN_TIMESTAMP, + TIME_GRANULARITY) # AfterCount is experimental. No backwards compatibility guarantees. @@ -64,14 +62,13 @@ class AccumulationMode(object): # RETRACTING = 3 -class _StateTag(object): +class _StateTag(with_metaclass(ABCMeta, object)): """An identifier used to store and retrieve typed, combinable state. The given tag must be unique for this stage. If CombineFn is None then all elements will be returned as a list, otherwise the given CombineFn will be applied (possibly incrementally and eagerly) when adding elements. """ - __metaclass__ = ABCMeta def __init__(self, tag): self.tag = tag @@ -132,12 +129,11 @@ def with_prefix(self, prefix): # pylint: disable=unused-argument # TODO(robertwb): Provisional API, Java likely to change as well. -class TriggerFn(object): +class TriggerFn(with_metaclass(ABCMeta, object)): """A TriggerFn determines when window (panes) are emitted. See https://beam.apache.org/documentation/programming-guide/#triggers """ - __metaclass__ = ABCMeta @abstractmethod def on_element(self, element, window, context): @@ -452,9 +448,7 @@ def to_runner_api(self, context): subtrigger=self.underlying.to_runner_api(context))) -class _ParallelTriggerFn(TriggerFn): - - __metaclass__ = ABCMeta +class _ParallelTriggerFn(with_metaclass(ABCMeta, TriggerFn)): def __init__(self, *triggers): self.triggers = triggers @@ -674,14 +668,12 @@ def clear_state(self, tag): # pylint: disable=unused-argument -class SimpleState(object): +class SimpleState(with_metaclass(ABCMeta, object)): """Basic state storage interface used for triggering. Only timers must hold the watermark (by their timestamp). """ - __metaclass__ = ABCMeta - @abstractmethod def set_timer(self, window, name, time_domain, timestamp): pass @@ -793,10 +785,10 @@ def merge(self, to_be_merged, merge_result): self._persist_window_ids() def known_windows(self): - return self.window_ids.keys() + return list(self.window_ids.keys()) def get_window(self, window_id): - for window, ids in self.window_ids.items(): + for window, ids in list(self.window_ids.items()): if window_id in ids: return window raise ValueError('No window for %s' % window_id) @@ -817,7 +809,9 @@ def _get_next_counter(self): if not self.window_ids: self.counter = 0 elif self.counter is None: - self.counter = max(k for ids in self.window_ids.values() for k in ids) + self.counter = max(k + for ids in list(self.window_ids.values()) + for k in ids) self.counter += 1 return self.counter @@ -846,11 +840,9 @@ def create_trigger_driver(windowing, is_batch=False, phased_combine_fn=None): return driver -class TriggerDriver(object): +class TriggerDriver(with_metaclass(ABCMeta, object)): """Breaks a series of bundle and timer firings into window (pane)s.""" - __metaclass__ = ABCMeta - @abstractmethod def process_elements(self, state, windowed_values, output_watermark): pass @@ -883,7 +875,7 @@ def __eq__(self, other): if isinstance(other, collections.Iterable): return all( a == b - for a, b in itertools.izip_longest(self, other, fillvalue=object())) + for a, b in itertools.zip_longest(self, other, fillvalue=object())) else: return NotImplemented @@ -961,7 +953,7 @@ def process_elements(self, state, windowed_values, output_watermark): # First handle merging. if self.is_merging: old_windows = set(state.known_windows()) - all_windows = old_windows.union(windows_to_elements.keys()) + all_windows = old_windows.union(list(windows_to_elements.keys())) if all_windows != old_windows: merged_away = {} @@ -980,7 +972,7 @@ def merge(_, to_be_merged, merge_result): # pylint: disable=no-self-argument self.window_fn.merge(TriggerMergeContext(all_windows)) merged_windows_to_elements = collections.defaultdict(list) - for window, values in windows_to_elements.items(): + for window, values in list(windows_to_elements.items()): while window in merged_away: window = merged_away[window] merged_windows_to_elements[window].extend(values) @@ -990,7 +982,7 @@ def merge(_, to_be_merged, merge_result): # pylint: disable=no-self-argument state.clear_state(window, self.WATERMARK_HOLD) # Next handle element adding. - for window, elements in windows_to_elements.items(): + for window, elements in list(windows_to_elements.items()): if state.get_state(window, self.TOMBSTONE): continue # Add watermark hold. @@ -1133,7 +1125,7 @@ def get_and_clear_timers(self, watermark=MAX_TIMESTAMP): def get_earliest_hold(self): earliest_hold = MAX_TIMESTAMP - for unused_window, tagged_states in self.state.iteritems(): + for unused_window, tagged_states in self.state.items(): # TODO(BEAM-2519): currently, this assumes that the watermark hold tag is # named "watermark". This is currently only true because the only place # watermark holds are set is in the GeneralTriggerDriver, where we use @@ -1146,5 +1138,5 @@ def get_earliest_hold(self): def __repr__(self): state_str = '\n'.join('%s: %s' % (key, dict(state)) - for key, state in self.state.items()) + for key, state in list(self.state.items())) return 'timers: %s\nstate: %s' % (dict(self.timers), state_str) diff --git a/sdks/python/apache_beam/transforms/trigger_test.py b/sdks/python/apache_beam/transforms/trigger_test.py index 1ae1f0250320..a0db26787883 100644 --- a/sdks/python/apache_beam/transforms/trigger_test.py +++ b/sdks/python/apache_beam/transforms/trigger_test.py @@ -21,6 +21,7 @@ import os.path import pickle import unittest +from builtins import range, zip import yaml @@ -30,25 +31,16 @@ from apache_beam.testing.util import assert_that, equal_to from apache_beam.transforms import trigger from apache_beam.transforms.core import Windowing -from apache_beam.transforms.trigger import AccumulationMode -from apache_beam.transforms.trigger import AfterAll -from apache_beam.transforms.trigger import AfterCount -from apache_beam.transforms.trigger import AfterEach -from apache_beam.transforms.trigger import AfterAny -from apache_beam.transforms.trigger import AfterWatermark -from apache_beam.transforms.trigger import DefaultTrigger -from apache_beam.transforms.trigger import GeneralTriggerDriver -from apache_beam.transforms.trigger import InMemoryUnmergedState -from apache_beam.transforms.trigger import Repeatedly -from apache_beam.transforms.trigger import TriggerFn -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.window import IntervalWindow -from apache_beam.transforms.window import MIN_TIMESTAMP -from apache_beam.transforms.window import TimestampCombiner -from apache_beam.transforms.window import Sessions -from apache_beam.transforms.window import TimestampedValue -from apache_beam.transforms.window import WindowedValue -from apache_beam.transforms.window import WindowFn +from apache_beam.transforms.trigger import (AccumulationMode, AfterAll, + AfterAny, AfterCount, AfterEach, + AfterWatermark, DefaultTrigger, + GeneralTriggerDriver, + InMemoryUnmergedState, Repeatedly, + TriggerFn) +from apache_beam.transforms.window import (MIN_TIMESTAMP, FixedWindows, + IntervalWindow, Sessions, + TimestampCombiner, TimestampedValue, + WindowedValue, WindowFn) class CustomTimestampingFixedWindowsWindowFn(FixedWindows): @@ -164,14 +156,14 @@ def test_fixed_watermark_with_early_late(self): AfterWatermark(early=AfterCount(3), late=AfterCount(2)), AccumulationMode.DISCARDING, - zip(range(9), 'abcdefghi'), + list(zip(list(range(9)), 'abcdefghi')), {IntervalWindow(0, 100): [ set('abcd'), set('efgh'), # early set('i'), # on time set('vw'), set('xy') # late ]}, 2, - late_data=zip(range(5), 'vwxyz')) + late_data=list(zip(list(range(5)), 'vwxyz'))) def test_sessions_watermark_with_early_late(self): self.run_trigger_simple( @@ -238,7 +230,7 @@ def test_repeatedly_after_first(self): FixedWindows(100), # pyformat break Repeatedly(AfterAny(AfterCount(3), AfterWatermark())), AccumulationMode.ACCUMULATING, - zip(range(7), 'abcdefg'), + list(zip(list(range(7)), 'abcdefg')), {IntervalWindow(0, 100): [ set('abc'), set('abcdef'), @@ -247,7 +239,7 @@ def test_repeatedly_after_first(self): set('abcdefgxy'), set('abcdefgxyz')]}, 1, - late_data=zip(range(3), 'xyz')) + late_data=list(zip(list(range(3)), 'xyz'))) def test_sessions_after_all(self): self.run_trigger_simple( @@ -355,7 +347,7 @@ def test_sessions_after_each(self): Sessions(10), # pyformat break AfterEach(AfterCount(2), AfterCount(3)), AccumulationMode.ACCUMULATING, - zip(range(10), 'abcdefghij'), + list(zip(list(range(10)), 'abcdefghij')), {IntervalWindow(0, 11): [set('ab')], IntervalWindow(0, 15): [set('abcdef')]}, 2) @@ -364,7 +356,7 @@ def test_sessions_after_each(self): Sessions(10), # pyformat break Repeatedly(AfterEach(AfterCount(2), AfterCount(3))), AccumulationMode.ACCUMULATING, - zip(range(10), 'abcdefghij'), + list(zip(list(range(10)), 'abcdefghij')), {IntervalWindow(0, 11): [set('ab')], IntervalWindow(0, 15): [set('abcdef')], IntervalWindow(0, 17): [set('abcdefgh')]}, @@ -379,7 +371,7 @@ def test_picklable_output(self): pickle.dumps(unpicklable) for unwindowed in driver.process_elements(None, unpicklable, None): self.assertEqual(pickle.loads(pickle.dumps(unwindowed)).value, - range(10)) + list(range(10))) class RunnerApiTest(unittest.TestCase): @@ -402,22 +394,29 @@ def test_trigger_encoding(self): class TriggerPipelineTest(unittest.TestCase): def test_after_count(self): + + def make_time_stamped_value(k_t): + return TimestampedValue((k_t[0], k_t[1]), k_t[1]) + + def format_result(k_v): + return ('%s-%s' % (k_v[0], len(k_v[1])), set(k_v[1])) + with TestPipeline() as p: result = (p | beam.Create([1, 2, 3, 4, 5, 10, 11]) | beam.FlatMap(lambda t: [('A', t), ('B', t + 5)]) - | beam.Map(lambda (k, t): TimestampedValue((k, t), t)) + | beam.Map(make_time_stamped_value) | beam.WindowInto(FixedWindows(10), trigger=AfterCount(3), accumulation_mode=AccumulationMode.DISCARDING) | beam.GroupByKey() - | beam.Map(lambda (k, v): ('%s-%s' % (k, len(v)), set(v)))) + | beam.Map(format_result)) assert_that(result, equal_to( - { + iter({ 'A-5': {1, 2, 3, 4, 5}, # A-10, A-11 never emitted due to AfterCount(3) never firing. 'B-4': {6, 7, 8, 9}, 'B-3': {10, 15, 16}, - }.iteritems())) + }.items()))) class TranscriptTest(unittest.TestCase): @@ -463,7 +462,7 @@ def split_args(s): args = [] start = 0 depth = 0 - for ix in xrange(len(s)): + for ix in range(len(s)): c = s[ix] if c in '({[': depth += 1 @@ -546,7 +545,7 @@ def fire_timers(): for line in spec['transcript']: - action, params = line.items()[0] + action, params = list(line.items())[0] if action != 'expect': # Fail if we have output that was not expected in the transcript. diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index a7484acb2cc7..2b3bdf110322 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -20,13 +20,10 @@ from __future__ import absolute_import -from apache_beam.transforms.core import CombinePerKey -from apache_beam.transforms.core import Flatten -from apache_beam.transforms.core import GroupByKey -from apache_beam.transforms.core import Map -from apache_beam.transforms.ptransform import PTransform -from apache_beam.transforms.ptransform import ptransform_fn +from builtins import range +from apache_beam.transforms.core import CombinePerKey, Flatten, GroupByKey, Map +from apache_beam.transforms.ptransform import PTransform, ptransform_fn __all__ = [ 'CoGroupByKey', @@ -86,12 +83,12 @@ def __init__(self, **kwargs): super(CoGroupByKey, self).__init__() self.pipeline = kwargs.pop('pipeline', None) if kwargs: - raise ValueError('Unexpected keyword arguments: %s' % kwargs.keys()) + raise ValueError('Unexpected keyword arguments: %s' % list(kwargs.keys())) def _extract_input_pvalues(self, pvalueish): try: # If this works, it's a dict. - return pvalueish, tuple(pvalueish.viewvalues()) + return pvalueish, tuple(pvalueish.values()) except AttributeError: pcolls = tuple(pvalueish) return pcolls, pcolls @@ -99,14 +96,16 @@ def _extract_input_pvalues(self, pvalueish): def expand(self, pcolls): """Performs CoGroupByKey on argument pcolls; see class docstring.""" # For associating values in K-V pairs with the PCollections they came from. - def _pair_tag_with_value((key, value), tag): + def _pair_tag_with_value(xxx_todo_changeme, tag): + (key, value) = xxx_todo_changeme return (key, (tag, value)) # Creates the key, value pairs for the output PCollection. Values are either # lists or dicts (per the class docstring), initialized by the result of # result_ctor(result_ctor_arg). - def _merge_tagged_vals_under_key((key, grouped), result_ctor, + def _merge_tagged_vals_under_key(xxx_todo_changeme3, result_ctor, result_ctor_arg): + (key, grouped) = xxx_todo_changeme3 result_value = result_ctor(result_ctor_arg) for tag, value in grouped: result_value[tag].append(value) @@ -116,15 +115,15 @@ def _merge_tagged_vals_under_key((key, grouped), result_ctor, # If pcolls is a dict, we turn it into (tag, pcoll) pairs for use in the # general-purpose code below. The result value constructor creates dicts # whose keys are the tags. - result_ctor_arg = pcolls.keys() + result_ctor_arg = list(pcolls.keys()) result_ctor = lambda tags: dict((tag, []) for tag in tags) - pcolls = pcolls.items() + pcolls = list(pcolls.items()) except AttributeError: # Otherwise, pcolls is a list/tuple, so we turn it into (index, pcoll) # pairs. The result value constructor makes tuples with len(pcolls) slots. pcolls = list(enumerate(pcolls)) result_ctor_arg = len(pcolls) - result_ctor = lambda size: tuple([] for _ in xrange(size)) + result_ctor = lambda size: tuple([] for _ in range(size)) # Check input PCollections for PCollection-ness, and that they all belong # to the same pipeline. @@ -142,17 +141,17 @@ def _merge_tagged_vals_under_key((key, grouped), result_ctor, def Keys(label='Keys'): # pylint: disable=invalid-name """Produces a PCollection of first elements of 2-tuples in a PCollection.""" - return label >> Map(lambda (k, v): k) + return label >> Map(lambda k_v: k_v[0]) def Values(label='Values'): # pylint: disable=invalid-name """Produces a PCollection of second elements of 2-tuples in a PCollection.""" - return label >> Map(lambda (k, v): v) + return label >> Map(lambda k_v1: k_v1[1]) def KvSwap(label='KvSwap'): # pylint: disable=invalid-name """Produces a PCollection reversing 2-tuples in a PCollection.""" - return label >> Map(lambda (k, v): (v, k)) + return label >> Map(lambda k_v2: (k_v2[1], k_v2[0])) @ptransform_fn diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py index 458fb747eb22..beb7ced000a1 100644 --- a/sdks/python/apache_beam/transforms/window.py +++ b/sdks/python/apache_beam/transforms/window.py @@ -50,23 +50,21 @@ from __future__ import absolute_import import abc +from builtins import object, range -from google.protobuf import duration_pb2 -from google.protobuf import timestamp_pb2 +from future.utils import with_metaclass +from google.protobuf import duration_pb2, timestamp_pb2 +from past.builtins import cmp from apache_beam.coders import coders -from apache_beam.portability.api import beam_runner_api_pb2 -from apache_beam.portability.api import standard_window_fns_pb2 +from apache_beam.portability.api import (beam_runner_api_pb2, + standard_window_fns_pb2) from apache_beam.transforms import timeutil -from apache_beam.utils import proto_utils -from apache_beam.utils import urns -from apache_beam.utils.timestamp import Duration -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.utils.timestamp import Timestamp +from apache_beam.utils import proto_utils, urns +from apache_beam.utils.timestamp import (MAX_TIMESTAMP, MIN_TIMESTAMP, + Duration, Timestamp) from apache_beam.utils.windowed_value import WindowedValue - __all__ = [ 'TimestampCombiner', 'WindowFn', @@ -107,11 +105,9 @@ def get_impl(timestamp_combiner, window_fn): raise ValueError('Invalid TimestampCombiner: %s.' % timestamp_combiner) -class WindowFn(urns.RunnerApiFn): +class WindowFn(with_metaclass(abc.ABCMeta, urns.RunnerApiFn)): """An abstract windowing function defining a basic assign and merge.""" - __metaclass__ = abc.ABCMeta - class AssignContext(object): """Context passed to WindowFn.assign().""" diff --git a/sdks/python/apache_beam/transforms/window_test.py b/sdks/python/apache_beam/transforms/window_test.py index 977a364ad883..13080dbef169 100644 --- a/sdks/python/apache_beam/transforms/window_test.py +++ b/sdks/python/apache_beam/transforms/window_test.py @@ -16,41 +16,33 @@ # """Unit tests for the windowing classes.""" +from __future__ import division import unittest +from builtins import range + +from past.utils import old_div from apache_beam.runners import pipeline_context from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, equal_to -from apache_beam.transforms import CombinePerKey -from apache_beam.transforms import combiners -from apache_beam.transforms import core -from apache_beam.transforms import Create -from apache_beam.transforms import GroupByKey -from apache_beam.transforms import Map -from apache_beam.transforms import WindowInto +from apache_beam.transforms import (CombinePerKey, Create, GroupByKey, Map, + WindowInto, combiners, core) from apache_beam.transforms.core import Windowing -from apache_beam.transforms.trigger import AccumulationMode -from apache_beam.transforms.trigger import AfterCount -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.window import GlobalWindow -from apache_beam.transforms.window import GlobalWindows -from apache_beam.transforms.window import IntervalWindow -from apache_beam.transforms.window import TimestampCombiner -from apache_beam.transforms.window import Sessions -from apache_beam.transforms.window import SlidingWindows -from apache_beam.transforms.window import TimestampedValue -from apache_beam.transforms.window import WindowedValue -from apache_beam.transforms.window import WindowFn -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP +from apache_beam.transforms.trigger import AccumulationMode, AfterCount +from apache_beam.transforms.window import (FixedWindows, GlobalWindow, + GlobalWindows, IntervalWindow, + Sessions, SlidingWindows, + TimestampCombiner, TimestampedValue, + WindowedValue, WindowFn) +from apache_beam.utils.timestamp import MAX_TIMESTAMP, MIN_TIMESTAMP def context(element, timestamp): return WindowFn.AssignContext(timestamp, element) -sort_values = Map(lambda (k, vs): (k, sorted(vs))) +sort_values = Map(lambda k_vs: (k_vs[0], sorted(k_vs[1]))) class ReifyWindowsFn(core.DoFn): @@ -194,7 +186,7 @@ def test_timestamped_value(self): with TestPipeline() as p: result = (p | 'start' >> Create([(k, k) for k in range(10)]) - | Map(lambda (x, t): TimestampedValue(x, t)) + | Map(lambda x_t: TimestampedValue(x_t[0], x_t[1])) | 'w' >> WindowInto(FixedWindows(5)) | Map(lambda v: ('key', v)) | GroupByKey()) @@ -205,7 +197,7 @@ def test_rewindow(self): with TestPipeline() as p: result = (p | Create([(k, k) for k in range(10)]) - | Map(lambda (x, t): TimestampedValue(x, t)) + | Map(lambda x_t1: TimestampedValue(x_t1[0], x_t1[1])) | 'window' >> WindowInto(SlidingWindows(period=2, size=6)) # Per the model, each element is now duplicated across # three windows. Rewindowing must preserve this duplication. @@ -231,11 +223,11 @@ def test_timestamped_with_combiners(self): # Now there are values 5 ms apart and since Map propagates the # windowing function from input to output the output PCollection # will have elements falling into different 5ms windows. - | Map(lambda (x, t): TimestampedValue(x, t)) + | Map(lambda x_t2: TimestampedValue(x_t2[0], x_t2[1])) # We add a 'key' to each value representing the index of the # window. This is important since there is no guarantee of # order for the elements of a PCollection. - | Map(lambda v: (v / 5, v))) + | Map(lambda v: (old_div(v, 5), v))) # Sum all elements associated with a key and window. Although it # is called CombinePerKey it is really CombinePerKeyAndWindow the # same way GroupByKey is really GroupByKeyAndWindow. diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index 50f0debb0a70..dc4d333198f1 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -20,7 +20,6 @@ import unittest import apache_beam as beam - from apache_beam.io import iobase from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, is_empty diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py index 3f5b4c9c479b..c0fd9385238a 100644 --- a/sdks/python/apache_beam/typehints/decorators.py +++ b/sdks/python/apache_beam/typehints/decorators.py @@ -85,14 +85,13 @@ def foo((a, b)): import inspect import types +from builtins import next, object, zip -from apache_beam.typehints import native_type_compatibility -from apache_beam.typehints import typehints -from apache_beam.typehints.typehints import check_constraint -from apache_beam.typehints.typehints import CompositeTypeHintError -from apache_beam.typehints.typehints import SimpleTypeHintError -from apache_beam.typehints.typehints import validate_composite_type_param - +from apache_beam.typehints import native_type_compatibility, typehints +from apache_beam.typehints.typehints import (CompositeTypeHintError, + SimpleTypeHintError, + check_constraint, + validate_composite_type_param) __all__ = [ 'with_input_types', @@ -118,7 +117,7 @@ def getargspec(func): try: return _original_getargspec(func) except TypeError: - if isinstance(func, (type, types.ClassType)): + if isinstance(func, type): argspec = getargspec(func.__init__) del argspec.args[0] return argspec @@ -176,7 +175,7 @@ def with_defaults(self, hints): return IOTypeHints(self.input_types or hints.input_types, self.output_types or hints.output_types) - def __nonzero__(self): + def __bool__(self): return bool(self.input_types or self.output_types) def __repr__(self): @@ -262,7 +261,7 @@ def getcallargs_forhints(func, *typeargs, **typekwargs): packed_typeargs += list(typeargs[len(packed_typeargs):]) try: callargs = inspect.getcallargs(func, *packed_typeargs, **typekwargs) - except TypeError, e: + except TypeError as e: raise TypeCheckError(e) if argspec.defaults: # Declare any default arguments to be Any. @@ -568,7 +567,7 @@ def __getattr__(self, attr): return self.__iter__() return getattr(self.internal_gen, attr) - def next(self): + def __next__(self): next_val = next(self.internal_gen) self.interleave_func(next_val) return next_val diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index 26c584e380d2..6a0e03714284 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -19,6 +19,8 @@ import collections import typing +from builtins import next, range + from apache_beam.typehints import typehints # Describes an entry in the type map in convert_to_beam_type. @@ -137,7 +139,7 @@ def convert_to_beam_type(typ): if _len_arg(typ) != arity: raise ValueError('expecting type %s to have arity %d, had arity %d ' 'instead' % (str(typ), arity, _len_arg(typ))) - typs = [convert_to_beam_type(_get_arg(typ, i)) for i in xrange(arity)] + typs = [convert_to_beam_type(_get_arg(typ, i)) for i in range(arity)] if arity == 0: # Nullary types (e.g. Any) don't accept empty tuples as arguments. return matched_entry.beam_type @@ -160,6 +162,6 @@ def convert_to_beam_types(args): a dictionary with the same keys, and values which have been converted. """ if isinstance(args, dict): - return {k: convert_to_beam_type(v) for k, v in args.iteritems()} + return {k: convert_to_beam_type(v) for k, v in args.items()} else: return [convert_to_beam_type(v) for v in args] diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py index 0ff2b3b5e58d..faca02650a98 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py @@ -17,12 +17,12 @@ """Test for Beam type compatibility library.""" +import typing import unittest +from builtins import object -import typing +from apache_beam.typehints import native_type_compatibility, typehints -from apache_beam.typehints import typehints -from apache_beam.typehints import native_type_compatibility _TestNamedTuple = typing.NamedTuple('_TestNamedTuple', [('age', int), ('name', bytes)]) diff --git a/sdks/python/apache_beam/typehints/opcodes.py b/sdks/python/apache_beam/typehints/opcodes.py index 83f444c6c53b..23f3fd2b87b0 100644 --- a/sdks/python/apache_beam/typehints/opcodes.py +++ b/sdks/python/apache_beam/typehints/opcodes.py @@ -26,11 +26,16 @@ For internal use only; no backwards-compatibility guarantees. """ +from __future__ import absolute_import, division + import types +from functools import reduce + +from past.utils import old_div -from trivial_inference import union, element_type, Const, BoundMethod -import typehints -from typehints import Any, Dict, Iterable, List, Tuple, Union +from . import typehints +from .trivial_inference import BoundMethod, Const, element_type, union +from .typehints import Any, Dict, Iterable, List, Tuple, Union def pop_one(state, unused_arg): @@ -136,7 +141,7 @@ def binary_true_divide(state, unused_arg): def binary_subscr(state, unused_arg): tos = state.stack.pop() - if tos in (str, unicode): + if tos in (str, str): out = tos else: out = element_type(tos) @@ -254,7 +259,7 @@ def load_attr(state, arg): name = state.get_name(arg) if isinstance(o, Const) and hasattr(o.value, name): state.stack.append(Const(getattr(o.value, name))) - elif (isinstance(o, (type, types.ClassType)) + elif (isinstance(o, type) and isinstance(getattr(o, name, None), types.MethodType)): state.stack.append(Const(BoundMethod(getattr(o, name)))) else: @@ -311,7 +316,7 @@ def load_deref(state, arg): def call_function(state, arg, has_var=False, has_kw=False): # TODO(robertwb): Recognize builtins and dataflow objects # (especially special return values). - pop_count = (arg & 0xF) + (arg & 0xF0) / 8 + 1 + has_var + has_kw + pop_count = (arg & 0xF) + old_div((arg & 0xF0), 8) + 1 + has_var + has_kw state.stack[-pop_count:] = [Any] diff --git a/sdks/python/apache_beam/typehints/trivial_inference.py b/sdks/python/apache_beam/typehints/trivial_inference.py index c7405963f161..db629bb23375 100644 --- a/sdks/python/apache_beam/typehints/trivial_inference.py +++ b/sdks/python/apache_beam/typehints/trivial_inference.py @@ -19,15 +19,23 @@ For internal use only; no backwards-compatibility guarantees. """ -import __builtin__ +from __future__ import absolute_import, division, print_function + +import builtins +from builtins import object, zip import collections import dis import pprint import sys import types +from functools import reduce + +from future import standard_library +from past.utils import old_div + +from apache_beam.typehints import Any, typehints -from apache_beam.typehints import Any -from apache_beam.typehints import typehints +standard_library.install_aliases() class TypeInferenceError(ValueError): @@ -103,7 +111,10 @@ class FrameState(object): def __init__(self, f, local_vars=None, stack=()): self.f = f - self.co = f.func_code + if sys.version_info >= 3: + self.co = f.__code__ + else: + self.co = f.func_code self.vars = list(local_vars) self.stack = list(stack) @@ -120,14 +131,14 @@ def closure_type(self, i): ncellvars = len(self.co.co_cellvars) if i < ncellvars: return Any - return Const(self.f.func_closure[i - ncellvars].cell_contents) + return Const(self.f.__closure__[i - ncellvars].cell_contents) def get_global(self, i): name = self.get_name(i) - if name in self.f.func_globals: - return Const(self.f.func_globals[name]) - if name in __builtin__.__dict__: - return Const(__builtin__.__dict__[name]) + if name in self.f.__globals__: + return Const(self.f.__globals__[name]) + if name in builtins.__dict__: + return Const(builtins.__dict__[name]) return Any def get_name(self, i): @@ -221,20 +232,22 @@ def infer_return_type(c, input_types, debug=False, depth=5): A TypeConstraint that that the return value of this function will (likely) satisfy given the specified inputs. """ + if debug: + print("Infering type on {0} for inputs {1}".format(c, input_types)) try: if hashable(c) and c in known_return_types: return known_return_types[c] elif isinstance(c, types.FunctionType): return infer_return_type_func(c, input_types, debug, depth) elif isinstance(c, types.MethodType): - if c.im_self is not None: - input_types = [Const(c.im_self)] + input_types - return infer_return_type_func(c.im_func, input_types, debug, depth) + if c.__self__ is not None: + input_types = [Const(c.__self__)] + input_types + return infer_return_type_func(c.__func__, input_types, debug, depth) elif isinstance(c, BoundMethod): - input_types = [c.unbound.im_class] + input_types + input_types = [c.unbound.__self__.__class__] + input_types return infer_return_type_func( - c.unbound.im_func, input_types, debug, depth) - elif isinstance(c, (type, types.ClassType)): + c.unbound.__func__, input_types, debug, depth) + elif isinstance(c, type): if c in typehints.DISALLOWED_PRIMITIVE_TYPES: return { list: typehints.List[Any], @@ -244,10 +257,16 @@ def infer_return_type(c, input_types, debug=False, depth=5): }[c] return c else: + if debug: + print("Concrete type {0} fell through.".format(c)) return Any - except TypeInferenceError: + except TypeInferenceError as e: + if debug: + print("Had type inference error {0}".format(e)) return Any - except Exception: + except Exception as e: + if debug: + print("Had more general error during inference {0}".format(e)) if debug: sys.stdout.flush() raise @@ -272,12 +291,14 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): TypeInferenceError: if no type can be inferred. """ if debug: - print - print f, id(f), input_types - import opcodes - simple_ops = dict((k.upper(), v) for k, v in opcodes.__dict__.items()) - - co = f.func_code + print() + print("Infering return type function {0} id {1} with input {2}" + .format(f, id(f), input_types)) + print() + from . import opcodes + simple_ops = dict((k.upper(), v) for k, v in list(opcodes.__dict__.items())) + + co = f.__code__ code = co.co_code end = len(code) pc = 0 @@ -299,45 +320,45 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): op = ord(code[pc]) if debug: - print '-->' if pc == last_pc else ' ', - print repr(pc).rjust(4), - print dis.opname[op].ljust(20), + print('-->' if pc == last_pc else ' ', end=' ') + print(repr(pc).rjust(4), end=' ') + print(dis.opname[op].ljust(20), end=' ') pc += 1 if op >= dis.HAVE_ARGUMENT: arg = ord(code[pc]) + ord(code[pc + 1]) * 256 + extended_arg extended_arg = 0 pc += 2 if op == dis.EXTENDED_ARG: - extended_arg = arg * 65536L + extended_arg = arg * 65536 if debug: - print str(arg).rjust(5), + print(str(arg).rjust(5), end=' ') if op in dis.hasconst: - print '(' + repr(co.co_consts[arg]) + ')', + print('(' + repr(co.co_consts[arg]) + ')', end=' ') elif op in dis.hasname: - print '(' + co.co_names[arg] + ')', + print('(' + co.co_names[arg] + ')', end=' ') elif op in dis.hasjrel: - print '(to ' + repr(pc + arg) + ')', + print('(to ' + repr(pc + arg) + ')', end=' ') elif op in dis.haslocal: - print '(' + co.co_varnames[arg] + ')', + print('(' + co.co_varnames[arg] + ')', end=' ') elif op in dis.hascompare: - print '(' + dis.cmp_op[arg] + ')', + print('(' + dis.cmp_op[arg] + ')', end=' ') elif op in dis.hasfree: if free is None: free = co.co_cellvars + co.co_freevars - print '(' + free[arg] + ')', + print('(' + free[arg] + ')', end=' ') # Acutally emulate the op. if state is None and states[start] is None: # No control reaches here (yet). if debug: - print + print() continue state |= states[start] opname = dis.opname[op] jmp = jmp_state = None if opname.startswith('CALL_FUNCTION'): - standard_args = (arg & 0xF) + (arg & 0xF0) / 8 + standard_args = (arg & 0xF) + old_div((arg & 0xF0), 8) var_args = 'VAR' in opname kw_args = 'KW' in opname pop_count = standard_args + var_args + kw_args + 1 @@ -358,7 +379,22 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): else: return_type = Any state.stack[-pop_count:] = [return_type] + elif (opname == 'BINARY_SUBSCR' + and isinstance(state.stack[1], Const) + and isinstance(state.stack[0], typehints.IndexableTypeConstraint)): + if debug: + print("Executing special case binary subscript") + idx = state.stack.pop() + src = state.stack.pop() + try: + state.stack.append(src._constraint_for_index(idx.value)) + except Exception as e: + if debug: + print("Exception {0} during special case indexing".format(e)) + state.stack.append(Any) elif opname in simple_ops: + if debug: + print("Executing simple op " + opname) simple_ops[opname](state, arg) elif opname == 'RETURN_VALUE': returns.add(state.stack[-1]) @@ -387,6 +423,8 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): jmp_state.stack.pop() state.stack.append(element_type(state.stack[-1])) else: + if debug: + print("unable to handle opname {0}".format(opname)) raise TypeInferenceError('unable to handle %s' % opname) if jmp is not None: @@ -398,9 +436,9 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): states[jmp] = new_state if debug: - print - print state - pprint.pprint(dict(item for item in states.items() if item[1])) + print() + print(state) + pprint.pprint(dict(item for item in list(states.items()) if item[1])) if yields: result = typehints.Iterable[reduce(union, Const.unwrap_all(yields))] @@ -408,5 +446,5 @@ def infer_return_type_func(f, input_types, debug=False, depth=0): result = reduce(union, Const.unwrap_all(returns)) if debug: - print f, id(f), input_types, '->', result + print(f, id(f), input_types, '->', result) return result diff --git a/sdks/python/apache_beam/typehints/trivial_inference_test.py b/sdks/python/apache_beam/typehints/trivial_inference_test.py index e7f451da11a6..e993ae1866ef 100644 --- a/sdks/python/apache_beam/typehints/trivial_inference_test.py +++ b/sdks/python/apache_beam/typehints/trivial_inference_test.py @@ -17,10 +17,9 @@ """Tests for apache_beam.typehints.trivial_inference.""" import unittest +from builtins import object - -from apache_beam.typehints import trivial_inference -from apache_beam.typehints import typehints +from apache_beam.typehints import trivial_inference, typehints global_int = 1 @@ -33,6 +32,11 @@ def assertReturnType(self, expected, f, inputs=()): def testIdentity(self): self.assertReturnType(int, lambda x: x, [int]) + def testIndexing(self): + self.assertReturnType(int, lambda x: x[0], [typehints.Tuple[int, str]]) + self.assertReturnType(str, lambda x: x[1], [typehints.Tuple[int, str]]) + self.assertReturnType(str, lambda x: x[1], [typehints.List[str]]) + def testTuples(self): self.assertReturnType( typehints.Tuple[typehints.Tuple[()], int], lambda x: ((), x), [int]) @@ -40,7 +44,8 @@ def testTuples(self): typehints.Tuple[str, int, float], lambda x: (x, 0, 1.0), [str]) def testUnpack(self): - def reverse((a, b)): + def reverse(xxx_todo_changeme): + (a, b) = xxx_todo_changeme return b, a any_tuple = typehints.Tuple[typehints.Any, typehints.Any] self.assertReturnType( diff --git a/sdks/python/apache_beam/typehints/typecheck.py b/sdks/python/apache_beam/typehints/typecheck.py index 89a5f5c7e2cf..1fbc01272e38 100644 --- a/sdks/python/apache_beam/typehints/typecheck.py +++ b/sdks/python/apache_beam/typehints/typecheck.py @@ -25,16 +25,17 @@ import sys import types +from past.builtins import basestring + from apache_beam.pvalue import TaggedOutput from apache_beam.transforms.core import DoFn from apache_beam.transforms.window import WindowedValue -from apache_beam.typehints.decorators import _check_instance_type -from apache_beam.typehints.decorators import getcallargs_forhints -from apache_beam.typehints.decorators import GeneratorWrapper -from apache_beam.typehints.decorators import TypeCheckError -from apache_beam.typehints.typehints import check_constraint -from apache_beam.typehints.typehints import CompositeTypeHintError -from apache_beam.typehints.typehints import SimpleTypeHintError +from apache_beam.typehints.decorators import (GeneratorWrapper, TypeCheckError, + _check_instance_type, + getcallargs_forhints) +from apache_beam.typehints.typehints import (CompositeTypeHintError, + SimpleTypeHintError, + check_constraint) class AbstractDoFnWrapper(DoFn): @@ -126,7 +127,7 @@ def wrapper(self, method, args, kwargs): def process(self, *args, **kwargs): if self._input_hints: actual_inputs = inspect.getcallargs(self._process_fn, *args, **kwargs) - for var, hint in self._input_hints.items(): + for var, hint in list(self._input_hints.items()): if hint is actual_inputs[var]: # self parameter continue diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test.py b/sdks/python/apache_beam/typehints/typed_pipeline_test.py index 59d1e1c5714e..4059941cc7c2 100644 --- a/sdks/python/apache_beam/typehints/typed_pipeline_test.py +++ b/sdks/python/apache_beam/typehints/typed_pipeline_test.py @@ -17,18 +17,21 @@ """Unit tests for the type-hint objects and decorators.""" import inspect -import unittest - +import sys import typing +import unittest import apache_beam as beam -from apache_beam import pvalue -from apache_beam import typehints +from apache_beam import pvalue, typehints from apache_beam.options.pipeline_options import OptionsContext from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that, equal_to from apache_beam.typehints import WithTypeHints +if sys.version_info[0] < 3: + basestring = str + + # These test often construct a pipeline as value | PTransform to test side # effects (e.g. errors). # pylint: disable=expression-not-assigned @@ -60,7 +63,7 @@ def test_non_function(self): [1, 2, 3] | beam.Map(str.upper) def test_loose_bounds(self): - @typehints.with_input_types(typehints.Union[int, float, long]) + @typehints.with_input_types(typehints.Union[int, float, int]) @typehints.with_output_types(basestring) def format_number(x): return '%g' % x @@ -103,14 +106,16 @@ class NativeTypesTest(unittest.TestCase): def test_good_main_input(self): @typehints.with_input_types(typing.Tuple[str, int]) - def munge((s, i)): + def munge(xxx_todo_changeme): + (s, i) = xxx_todo_changeme return (s + 's', i * 2) result = [('apple', 5), ('pear', 3)] | beam.Map(munge) self.assertEqual([('apples', 10), ('pears', 6)], sorted(result)) def test_bad_main_input(self): @typehints.with_input_types(typing.Tuple[str, str]) - def munge((s, i)): + def munge(xxx_todo_changeme1): + (s, i) = xxx_todo_changeme1 return (s + 's', i * 2) with self.assertRaises(typehints.TypeCheckError): [('apple', 5), ('pear', 3)] | beam.Map(munge) @@ -118,7 +123,8 @@ def munge((s, i)): def test_bad_main_output(self): @typehints.with_input_types(typing.Tuple[int, int]) @typehints.with_output_types(typing.Tuple[str, str]) - def munge((a, b)): + def munge(xxx_todo_changeme2): + (a, b) = xxx_todo_changeme2 return (str(a), str(b)) with self.assertRaises(typehints.TypeCheckError): [(5, 4), (3, 2)] | beam.Map(munge) | 'Again' >> beam.Map(munge) diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 98d399b82f3b..b9376949aa7a 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -62,10 +62,20 @@ 'type_check' method on each TypeConstraint. """ - import collections import copy +import sys import types +from builtins import zip + +from future.utils import with_metaclass + +# Keep object around for safety +base_object = object +if sys.version_info[0] >= 3: + basestring = str +else: + from builtins import object __all__ = [ @@ -185,7 +195,17 @@ def bind_type_variables(type_constraint, bindings): return type_constraint -class SequenceTypeConstraint(TypeConstraint): +class IndexableTypeConstraint(TypeConstraint): + """An internal common base-class for all type constraints with indexing. + E.G. SequenceTypeConstraint + Tuple's of fixed size. + """ + + def _constraint_for_index(self, idx): + """Returns the type at the given index.""" + raise NotImplementedError + + +class SequenceTypeConstraint(IndexableTypeConstraint): """A common base-class for all sequence related type-constraint classes. A sequence is defined as an arbitrary length homogeneous container type. Type @@ -215,6 +235,10 @@ def __hash__(self): def _inner_types(self): yield self.inner_type + def _constraint_for_index(self, idx): + """Returns the type at the given index.""" + return self.inner_type + def _consistent_with_check_(self, sub): return (isinstance(sub, self.__class__) and is_consistent_with(sub.inner_type, self.inner_type)) @@ -315,9 +339,11 @@ def validate_composite_type_param(type_param, error_msg_prefix): parameter for a :class:`CompositeTypeHint`. """ # Must either be a TypeConstraint instance or a basic Python type. - is_not_type_constraint = ( - not isinstance(type_param, (type, types.ClassType, TypeConstraint)) - and type_param is not None) + possible_classes = [type, TypeConstraint] + if sys.version_info[0] == 2: + possible_classes.append(types.ClassType) + is_not_type_constraint = (not isinstance(type_param, tuple(possible_classes)) + and type_param is not None) is_forbidden_type = (isinstance(type_param, type) and type_param in DISALLOWED_PRIMITIVE_TYPES) @@ -341,7 +367,7 @@ def _unified_repr(o): A qualified name for the passed Python object fit for string formatting. """ return repr(o) if isinstance( - o, (TypeConstraint, types.NoneType)) else o.__name__ + o, (TypeConstraint, type(None))) else o.__name__ def check_constraint(type_constraint, object_instance): @@ -492,7 +518,7 @@ def __getitem__(self, type_params): if Any in params: return Any elif len(params) == 1: - return iter(params).next() + return next(iter(params)) return self.UnionConstraint(params) @@ -547,7 +573,7 @@ def _consistent_with_check_(self, sub): for elem in sub.tuple_types) return super(TupleSequenceConstraint, self)._consistent_with_check_(sub) - class TupleConstraint(TypeConstraint): + class TupleConstraint(IndexableTypeConstraint): def __init__(self, type_params): self.tuple_types = tuple(type_params) @@ -567,6 +593,10 @@ def _inner_types(self): for t in self.tuple_types: yield t + def _constraint_for_index(self, idx): + """Returns the type at the given index.""" + return self.tuple_types[idx] + def _consistent_with_check_(self, sub): return (isinstance(sub, self.__class__) and len(sub.tuple_types) == len(self.tuple_types) @@ -620,6 +650,9 @@ def bind_type_variables(self, bindings): return self return Tuple[bound_tuple_types] + def __getitem__(self, index): + return self.tuple_types[index] + def __getitem__(self, type_params): ellipsis = False @@ -777,7 +810,7 @@ def type_check(self, dict_instance): 'type dict. %s is of type %s.' % (dict_instance, dict_instance.__class__.__name__)) - for key, value in dict_instance.iteritems(): + for key, value in dict_instance.items(): try: check_constraint(self.key_type, key) except CompositeTypeHintError as e: @@ -960,7 +993,8 @@ def __getitem__(self, type_param): IteratorTypeConstraint = IteratorHint.IteratorTypeConstraint -class WindowedTypeConstraint(TypeConstraint): +class WindowedTypeConstraint( + with_metaclass(GetitemConstructor, TypeConstraint)): """A type constraint for WindowedValue objects. Mostly for internal use. @@ -968,7 +1002,6 @@ class WindowedTypeConstraint(TypeConstraint): Attributes: inner_type: The type which the element should be an instance of. """ - __metaclass__ = GetitemConstructor def __init__(self, inner_type): self.inner_type = inner_type @@ -977,6 +1010,10 @@ def __eq__(self, other): return (isinstance(other, WindowedTypeConstraint) and self.inner_type == other.inner_type) + def __str__(self): + return "WindowedTypeConstraint {0} of type {1}".format( + self.__hash__(), self.inner_type) + def __hash__(self): return hash(self.inner_type) ^ 13 * hash(type(self)) @@ -1031,8 +1068,9 @@ class GeneratorHint(IteratorHint): list: List[Any], tuple: Tuple[Any, ...], set: Set[Any], + object: base_object, # Using None for the NoneType is a common convention. - None: type(None), + None: type(None) } diff --git a/sdks/python/apache_beam/typehints/typehints_test.py b/sdks/python/apache_beam/typehints/typehints_test.py index f1b92e096264..3ffb42344a13 100644 --- a/sdks/python/apache_beam/typehints/typehints_test.py +++ b/sdks/python/apache_beam/typehints/typehints_test.py @@ -19,22 +19,19 @@ import functools import inspect import unittest - +from builtins import next, object, range import apache_beam.typehints.typehints as typehints -from apache_beam.typehints import Any -from apache_beam.typehints import Tuple -from apache_beam.typehints import TypeCheckError -from apache_beam.typehints import Union -from apache_beam.typehints import with_input_types -from apache_beam.typehints import with_output_types -from apache_beam.typehints.decorators import _check_instance_type -from apache_beam.typehints.decorators import _interleave_type_check -from apache_beam.typehints.decorators import _positional_arg_hints -from apache_beam.typehints.decorators import get_type_hints -from apache_beam.typehints.decorators import getcallargs_forhints -from apache_beam.typehints.decorators import GeneratorWrapper +from apache_beam.typehints import (Any, Tuple, TypeCheckError, Union, + with_input_types, with_output_types) +from apache_beam.typehints.decorators import (GeneratorWrapper, + _check_instance_type, + _interleave_type_check, + _positional_arg_hints, + get_type_hints, + getcallargs_forhints) from apache_beam.typehints.typehints import is_consistent_with +from apache_beam.utils.test_utils import _rewrite_typehint_string def check_or_interleave(hint, value, var): @@ -54,7 +51,7 @@ def wrapper(*args, **kwargs): input_hints = getcallargs_forhints( f, *hints.input_types[0], **hints.input_types[1]) inputs = inspect.getcallargs(f, *args, **kwargs) - for var, hint in input_hints.items(): + for var, hint in list(input_hints.items()): value = inputs[var] new_value = check_or_interleave(hint, value, var) if new_value is not value: @@ -197,6 +194,8 @@ def test_union_hint_repr(self): str(hint), # Uses frozen set internally, so order not guaranteed. ['Union[str, DummyTestClass1]', + 'Union[DummyTestClass1, str]', + 'Union[str, DummyTestClass1]', 'Union[DummyTestClass1, str]'] ) @@ -220,7 +219,7 @@ def test_union_hint_enforcement_not_part_of_union(self): self.assertEqual("Union[float, int] type-constraint violated. Expected an " "instance of one of: ('float', 'int'), received str " "instead.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) class OptionalHintTestCase(TypeHintTestCase): @@ -333,7 +332,7 @@ def test_type_check_invalid_simple_types(self): 'type of element #0 in the passed tuple is incorrect.' ' Expected an instance of type str, instead received ' 'an instance of type int.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_check_invalid_composite_type(self): hint = typehints.Tuple[DummyTestClass1, DummyTestClass2] @@ -383,7 +382,7 @@ def test_type_check_invalid_simple_type_arbitrary_length(self): 'of element #2 in the passed tuple is incorrect. Expected ' 'an instance of type str, instead received an instance of ' 'type int.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_check_invalid_composite_type_arbitrary_length(self): hint = typehints.Tuple[typehints.List[int], ...] @@ -440,7 +439,7 @@ def test_enforce_list_type_constraint_invalid_simple_type(self): 'element #0 in the passed list is incorrect. Expected an ' 'instance of type int, instead received an instance of ' 'type str.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_enforce_list_type_constraint_invalid_composite_type(self): hint = typehints.List[typehints.Tuple[int, int]] @@ -454,7 +453,7 @@ def test_enforce_list_type_constraint_invalid_composite_type(self): 'violated. The type of element #0 in the passed tuple' ' is incorrect. Expected an instance of type int, ' 'instead received an instance of type str.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) class KVHintTestCase(TypeHintTestCase): @@ -473,8 +472,9 @@ def test_getitem_param_must_have_length_2(self): self.assertEqual("Length of parameters to a KV type-hint must be " "exactly 2. Passed parameters: (, , ), have a length of 3.", - e.exception.message) + "'str'>, ), have a" + " length of 3.", + _rewrite_typehint_string(e.exception.message)) def test_getitem_proxy_to_tuple(self): hint = typehints.KV[int, str] @@ -503,7 +503,7 @@ def test_getitem_param_must_have_length_2(self): self.assertEqual("Length of parameters to a Dict type-hint must be " "exactly 2. Passed parameters: (, , ), have a length of 3.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_key_type_must_be_valid_composite_param(self): with self.assertRaises(TypeError): @@ -525,7 +525,8 @@ def test_compatibility(self): def test_repr(self): hint3 = typehints.Dict[int, typehints.List[typehints.Tuple[str, str, str]]] - self.assertEqual('Dict[int, List[Tuple[str, str, str]]]', repr(hint3)) + self.assertEqual('Dict[int, List[Tuple[str, str, str]]]', + _rewrite_typehint_string(repr(hint3))) def test_type_checks_not_dict(self): hint = typehints.Dict[int, str] @@ -549,19 +550,19 @@ def test_type_check_invalid_key_type(self): 'instance is of the proper type, but differs in ' 'length from the hinted type. Expected a tuple of ' 'length 3, received a tuple of length 2.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_check_invalid_value_type(self): hint = typehints.Dict[str, typehints.Dict[int, str]] d = {'f': [1, 2, 3]} with self.assertRaises(TypeError) as e: hint.type_check(d) - self.assertEqual('Dict[str, Dict[int, str]] hint value-type constraint' - ' violated. All values should be of type ' + self.assertEqual('Dict[str, Dict[int, str]] hint value-type ' + 'constraint violated. All values should be of type ' 'Dict[int, str]. Instead: Dict type-constraint ' 'violated. All passed instances must be of type dict.' ' [1, 2, 3] is of type list.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_check_valid_simple_type(self): hint = typehints.Dict[int, str] @@ -669,7 +670,8 @@ def test_tuple_compatibility(self): def test_repr(self): hint = typehints.Iterable[typehints.Set[str]] - self.assertEqual('Iterable[Set[str]]', repr(hint)) + self.assertEqual('Iterable[Set[str]]', + _rewrite_typehint_string(repr(hint))) def test_type_check_must_be_iterable(self): with self.assertRaises(TypeError) as e: @@ -727,7 +729,7 @@ class GeneratorHintTestCase(TypeHintTestCase): def test_repr(self): hint = typehints.Iterator[typehints.Set[str]] - self.assertEqual('Iterator[Set[str]]', repr(hint)) + self.assertEqual('Iterator[Set[str]]', _rewrite_typehint_string(repr(hint))) def test_compatibility(self): self.assertCompatible(typehints.Iterator[int], typehints.Iterator[int]) @@ -747,7 +749,7 @@ def all_upper(s): 'hint type-constraint violated. Expected a iterator ' 'of type int. Instead received a iterator of type ' 'str.', - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_generator_argument_hint_invalid_yield_type(self): def wrong_yield_gen(): @@ -766,7 +768,7 @@ def increment(a): "hint type-constraint violated. Expected a iterator " "of type int. Instead received a iterator of type " "str.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) class TakesDecoratorTestCase(TypeHintTestCase): @@ -808,7 +810,7 @@ def foo(a): self.assertEqual("Type-hint for argument: 'a' violated. Expected an " "instance of , instead found an " "instance of .", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_composite_type_assertion(self): @check_type_hints @@ -824,7 +826,7 @@ def foo(a): "type-constraint violated. The type of element #0 in " "the passed list is incorrect. Expected an instance of " "type int, instead received an instance of type str.", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_valid_simple_type_arguments(self): @with_input_types(a=str) @@ -862,7 +864,7 @@ def sub(a, b): self.assertEqual("Type-hint for argument: 'b' violated. Expected an " "instance of , instead found an instance " "of .", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_valid_only_positional_arguments(self): @with_input_types(int, int) @@ -908,7 +910,7 @@ def foo(a): self.assertEqual("Type-hint for return type violated. Expected an " "instance of , instead found an instance " "of .", - e.exception.message) + _rewrite_typehint_string(e.exception.message)) def test_type_check_simple_type(self): @with_output_types(str) diff --git a/sdks/python/apache_beam/utils/annotations.py b/sdks/python/apache_beam/utils/annotations.py index 017dd6b81a4d..46c8b58c7e25 100644 --- a/sdks/python/apache_beam/utils/annotations.py +++ b/sdks/python/apache_beam/utils/annotations.py @@ -61,8 +61,7 @@ def exp_multiply(arg1, arg2): """ import warnings -from functools import partial -from functools import wraps +from functools import partial, wraps # Produce only the first occurrence of matching warnings regardless of # location per line of execution. Since the number of lines of execution @@ -109,3 +108,16 @@ def inner(*args, **kwargs): current=None, extra_message=None) experimental = partial(annotate, label='experimental', current=None, since=None, extra_message=None) + + +def ignore_unicode_prefix(f): + """ + Ignore the 'u' prefix of string in doc tests, to make it works + in both python 2 and 3 + """ + if sys.version >= '3': + # the representation of unicode string in Python 3 does not have prefix 'u', + # so remove the prefix 'u' for doc tests + literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) + f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) + return f diff --git a/sdks/python/apache_beam/utils/annotations_test.py b/sdks/python/apache_beam/utils/annotations_test.py index 32af8a991a79..94ed058b22d5 100644 --- a/sdks/python/apache_beam/utils/annotations_test.py +++ b/sdks/python/apache_beam/utils/annotations_test.py @@ -17,8 +17,8 @@ import unittest import warnings -from apache_beam.utils.annotations import deprecated -from apache_beam.utils.annotations import experimental + +from apache_beam.utils.annotations import deprecated, experimental class AnnotationTests(unittest.TestCase): diff --git a/sdks/python/apache_beam/utils/counters.py b/sdks/python/apache_beam/utils/counters.py index 08685aae2759..d7c79962cf95 100644 --- a/sdks/python/apache_beam/utils/counters.py +++ b/sdks/python/apache_beam/utils/counters.py @@ -25,6 +25,7 @@ from collections import namedtuple import threading +from builtins import hex, object from apache_beam.transforms import cy_combiners @@ -190,4 +191,4 @@ def get_counters(self): this method returns hence the returned iterable may be stale. """ with self._lock: - return self.counters.values() + return list(self.counters.values()) diff --git a/sdks/python/apache_beam/utils/plugin.py b/sdks/python/apache_beam/utils/plugin.py index 563b93c54c7d..31b440948cc9 100644 --- a/sdks/python/apache_beam/utils/plugin.py +++ b/sdks/python/apache_beam/utils/plugin.py @@ -21,6 +21,9 @@ """ +from builtins import object + + class BeamPlugin(object): """Plugin base class to be extended by dependent users such as FileSystem. Any instantiated subclass will be imported at worker startup time.""" diff --git a/sdks/python/apache_beam/utils/processes_test.py b/sdks/python/apache_beam/utils/processes_test.py index 920b621b1bbd..2dd45f44dc53 100644 --- a/sdks/python/apache_beam/utils/processes_test.py +++ b/sdks/python/apache_beam/utils/processes_test.py @@ -18,7 +18,6 @@ import unittest - import mock from apache_beam.utils import processes diff --git a/sdks/python/apache_beam/utils/profiler.py b/sdks/python/apache_beam/utils/profiler.py index a2c3f6ab1af6..0e2cc3e83d82 100644 --- a/sdks/python/apache_beam/utils/profiler.py +++ b/sdks/python/apache_beam/utils/profiler.py @@ -21,15 +21,20 @@ """ import cProfile +import io import logging import os import pstats -import StringIO import tempfile import time import warnings +from builtins import object from threading import Timer +from future import standard_library + +standard_library.install_aliases() + class Profile(object): """cProfile wrapper context for saving and logging profiler results.""" @@ -66,7 +71,7 @@ def __exit__(self, *args): os.remove(filename) if self.log_results: - s = StringIO.StringIO() + s = io.StringIO() self.stats = pstats.Stats( self.profile, stream=s).sort_stats(Profile.SORTBY) self.stats.print_stats() diff --git a/sdks/python/apache_beam/utils/proto_utils.py b/sdks/python/apache_beam/utils/proto_utils.py index d7693f3f7839..79110474fe77 100644 --- a/sdks/python/apache_beam/utils/proto_utils.py +++ b/sdks/python/apache_beam/utils/proto_utils.py @@ -17,8 +17,7 @@ """For internal use only; no backwards-compatibility guarantees.""" -from google.protobuf import any_pb2 -from google.protobuf import struct_pb2 +from google.protobuf import any_pb2, struct_pb2 def pack_Any(msg): @@ -61,7 +60,7 @@ def pack_Struct(**kwargs): """Returns a struct containing the values indicated by kwargs. """ msg = struct_pb2.Struct() - for key, value in kwargs.items(): + for key, value in list(kwargs.items()): msg[key] = value # pylint: disable=unsubscriptable-object, unsupported-assignment-operation return msg diff --git a/sdks/python/apache_beam/utils/retry.py b/sdks/python/apache_beam/utils/retry.py index 08223b3f1f7f..541d95f6ad16 100644 --- a/sdks/python/apache_beam/utils/retry.py +++ b/sdks/python/apache_beam/utils/retry.py @@ -30,6 +30,7 @@ import sys import time import traceback +from builtins import object, range from apache_beam.io.filesystem import BeamIOError @@ -80,7 +81,7 @@ def __init__(self, initial_delay_secs, num_retries, factor=2, fuzz=0.5, def __iter__(self): current_delay_secs = min(self._max_delay_secs, self._initial_delay_secs) - for _ in xrange(self._num_retries): + for _ in range(self._num_retries): fuzz_multiplier = 1 - self._fuzz + random.random() * self._fuzz yield current_delay_secs * fuzz_multiplier current_delay_secs = min( @@ -182,10 +183,16 @@ def wrapper(*args, **kwargs): exn_traceback = sys.exc_info()[2] try: try: - sleep_interval = retry_intervals.next() - except StopIteration: + sleep_interval = next(retry_intervals) + except StopIteration as inner: # Re-raise the original exception since we finished the retries. - raise exn, None, exn_traceback # pylint: disable=raising-bad-type + # Python 3 the traceback is in the exception, Python 2 no. + if sys.version >= "3": + raise exn + else: + exn.msg = 'Error during {0} caused by {1}'.format( + repr(inner), repr(exn)) + raise exn # pylint: disable=raising-bad-type logger( 'Retry with exponential backoff: waiting for %s seconds before ' diff --git a/sdks/python/apache_beam/utils/retry_test.py b/sdks/python/apache_beam/utils/retry_test.py index 1b03c835cd7f..5877159fa92f 100644 --- a/sdks/python/apache_beam/utils/retry_test.py +++ b/sdks/python/apache_beam/utils/retry_test.py @@ -18,6 +18,9 @@ """Unit tests for the retry module.""" import unittest +from builtins import object + +from apache_beam.utils import retry # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position @@ -29,9 +32,6 @@ # pylint: enable=wrong-import-order, wrong-import-position -from apache_beam.utils import retry - - class FakeClock(object): """A fake clock object implementing sleep() and recording calls.""" diff --git a/sdks/python/apache_beam/utils/test_utils.py b/sdks/python/apache_beam/utils/test_utils.py new file mode 100644 index 000000000000..d8809d12b7f6 --- /dev/null +++ b/sdks/python/apache_beam/utils/test_utils.py @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Basic test utils""" +import re + + +def _rewrite_typehint_string(type_hint_string): + first_pass = re.sub(r"class future.types.new(str|int).new(str|int)", + r'type \1', + type_hint_string) + second_pass = re.sub(r"future.types.new(str|int).new(str|int)", + r'\1', + first_pass) + return re.sub(r"new(str|int)", r'\1', second_pass) diff --git a/sdks/python/apache_beam/utils/timestamp.py b/sdks/python/apache_beam/utils/timestamp.py index b3e840ee284e..c370ab4111de 100644 --- a/sdks/python/apache_beam/utils/timestamp.py +++ b/sdks/python/apache_beam/utils/timestamp.py @@ -20,9 +20,13 @@ For internal use only; no backwards-compatibility guarantees. """ -from __future__ import absolute_import +from __future__ import absolute_import, division import datetime +from builtins import object + +from past.builtins import cmp +from past.utils import old_div class Timestamp(object): @@ -68,7 +72,7 @@ def __repr__(self): if micros < 0: sign = '-' micros = -micros - int_part = micros / 1000000 + int_part = old_div(micros, 1000000) frac_part = micros % 1000000 if frac_part: return 'Timestamp(%s%d.%06d)' % (sign, int_part, frac_part) @@ -86,11 +90,11 @@ def isoformat(self): def __float__(self): # Note that the returned value may have lost precision. - return float(self.micros) / 1000000 + return old_div(float(self.micros), 1000000) def __int__(self): # Note that the returned value may have lost precision. - return self.micros / 1000000 + return old_div(self.micros, 1000000) def __cmp__(self, other): # Allow comparisons between Duration and Timestamp values. @@ -160,7 +164,7 @@ def __repr__(self): if micros < 0: sign = '-' micros = -micros - int_part = micros / 1000000 + int_part = old_div(micros, 1000000) frac_part = micros % 1000000 if frac_part: return 'Duration(%s%d.%06d)' % (sign, int_part, frac_part) @@ -168,7 +172,7 @@ def __repr__(self): def __float__(self): # Note that the returned value may have lost precision. - return float(self.micros) / 1000000 + return old_div(float(self.micros), 1000000) def __cmp__(self, other): # Allow comparisons between Duration and Timestamp values. diff --git a/sdks/python/apache_beam/utils/timestamp_test.py b/sdks/python/apache_beam/utils/timestamp_test.py index 33229361b36f..04b19ed87a68 100644 --- a/sdks/python/apache_beam/utils/timestamp_test.py +++ b/sdks/python/apache_beam/utils/timestamp_test.py @@ -21,8 +21,7 @@ import unittest -from apache_beam.utils.timestamp import Duration -from apache_beam.utils.timestamp import Timestamp +from apache_beam.utils.timestamp import Duration, Timestamp class TimestampTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index 18959be87503..8e101fd80365 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -19,13 +19,13 @@ import abc import inspect +from builtins import object from google.protobuf import wrappers_pb2 from apache_beam.internal import pickler from apache_beam.utils import proto_utils - PICKLED_WINDOW_FN = "beam:windowfn:pickled_python:v0.1" GLOBAL_WINDOWS_FN = "beam:windowfn:global_windows:v0.1" FIXED_WINDOWS_FN = "beam:windowfn:fixed_windows:v0.1" diff --git a/sdks/python/apache_beam/utils/windowed_value.py b/sdks/python/apache_beam/utils/windowed_value.py index be2785432a14..dbf28a30a8ba 100644 --- a/sdks/python/apache_beam/utils/windowed_value.py +++ b/sdks/python/apache_beam/utils/windowed_value.py @@ -27,9 +27,11 @@ #cython: profile=True -from apache_beam.utils.timestamp import MAX_TIMESTAMP -from apache_beam.utils.timestamp import MIN_TIMESTAMP -from apache_beam.utils.timestamp import Timestamp +from builtins import object + +from past.builtins import cmp + +from apache_beam.utils.timestamp import MAX_TIMESTAMP, MIN_TIMESTAMP, Timestamp class WindowedValue(object): diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index a3d963d18d78..fa94d70b85d7 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -122,4 +122,4 @@ def _install_grpcio_tools_and_generate_proto_files(): if __name__ == '__main__': - generate_proto_files() \ No newline at end of file + generate_proto_files() diff --git a/sdks/python/generate_pydoc.sh b/sdks/python/generate_pydoc.sh index 662bd098e109..2967cfec1749 100755 --- a/sdks/python/generate_pydoc.sh +++ b/sdks/python/generate_pydoc.sh @@ -166,8 +166,8 @@ python $(type -p sphinx-build) -v -a -E -q target/docs/source \ -w "target/docs/sphinx-build.warnings.log" # Fail if there are errors or warnings in docs -! grep -q "ERROR:" target/docs/sphinx-build.warnings.log || exit 1 -! grep -q "WARNING:" target/docs/sphinx-build.warnings.log || exit 1 +! (grep -v future target/docs/sphinx-build.warnings.log | grep -q "ERROR:") || exit 1 +! (grep -v future target/docs/sphinx-build.warnings.log | grep -q "WARNING:") || exit 1 # Run tests for code samples, these can be: # - Code blocks using '.. testsetup::', '.. testcode::' and '.. testoutput::' @@ -176,9 +176,9 @@ python -msphinx -M doctest target/docs/source \ target/docs/_build -c target/docs/source \ -w "target/docs/sphinx-doctest.warnings.log" -# Fail if there are errors or warnings in docs -! grep -q "ERROR:" target/docs/sphinx-doctest.warnings.log || exit 1 -! grep -q "WARNING:" target/docs/sphinx-doctest.warnings.log || exit 1 +# Fail if there are errors or warnings in docs. We skip warnings/errors about future libs. +! (grep -v future target/docs/sphinx-doctest.warnings.log | grep -q "ERROR:") || exit 1 +! (grep -v future target/docs/sphinx-doctest.warnings.log | grep -q "WARNING:") || exit 1 # Message is useful only when this script is run locally. In a remote # test environment, this path will be removed when the test completes. diff --git a/sdks/python/run_postcommit.sh b/sdks/python/run_postcommit.sh index ddc3dc7de770..3bd465ac6cad 100755 --- a/sdks/python/run_postcommit.sh +++ b/sdks/python/run_postcommit.sh @@ -29,7 +29,7 @@ set -e set -v # pip install --user installation location. -LOCAL_PATH=$HOME/.local/bin/ +export LOCAL_PATH=$HOME/.local/bin/ # Remove any tox cache from previous workspace rm -rf sdks/python/target/.tox @@ -43,63 +43,4 @@ pip install tox --user # Tox runs unit tests in a virtual environment ${LOCAL_PATH}/tox -e ALL -c sdks/python/tox.ini -# Virtualenv for the rest of the script to run setup & e2e tests -${LOCAL_PATH}/virtualenv sdks/python -. sdks/python/bin/activate -cd sdks/python -pip install -e .[gcp,test] - -# Run wordcount in the Direct Runner and validate output. -echo ">>> RUNNING DIRECT RUNNER py-wordcount" -python -m apache_beam.examples.wordcount --output /tmp/py-wordcount-direct -# TODO: check that output file is generated for Direct Runner. - -# Run tests on the service. - -# Where to store integration test outputs. -GCS_LOCATION=gs://temp-storage-for-end-to-end-tests - -PROJECT=apache-beam-testing - -# Create a tarball -python setup.py sdist - -SDK_LOCATION=$(find dist/apache-beam-*.tar.gz) - -# Install test dependencies for ValidatesRunner tests. -echo "pyhamcrest" > postcommit_requirements.txt -echo "mock" >> postcommit_requirements.txt - -# Run ValidatesRunner tests on Google Cloud Dataflow service -echo ">>> RUNNING DATAFLOW RUNNER VALIDATESRUNNER TESTS" -python setup.py nosetests \ - --attr ValidatesRunner \ - --nocapture \ - --processes=4 \ - --process-timeout=900 \ - --test-pipeline-options=" \ - --runner=TestDataflowRunner \ - --project=$PROJECT \ - --staging_location=$GCS_LOCATION/staging-validatesrunner-test \ - --temp_location=$GCS_LOCATION/temp-validatesrunner-test \ - --sdk_location=$SDK_LOCATION \ - --requirements_file=postcommit_requirements.txt \ - --num_workers=1" - -# Run integration tests on the Google Cloud Dataflow service -# and validate that jobs finish successfully. -echo ">>> RUNNING TEST DATAFLOW RUNNER it tests" -python setup.py nosetests \ - --attr IT \ - --nocapture \ - --processes=4 \ - --process-timeout=900 \ - --test-pipeline-options=" \ - --runner=TestDataflowRunner \ - --project=$PROJECT \ - --staging_location=$GCS_LOCATION/staging-it \ - --temp_location=$GCS_LOCATION/temp-it \ - --output=$GCS_LOCATION/py-it-cloud/output \ - --sdk_location=$SDK_LOCATION \ - --num_workers=1 \ - --sleep_secs=20" +python post_commit.py diff --git a/sdks/python/setup.py b/sdks/python/setup.py index c13da8e326c8..10d5bddbf6da 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -42,6 +42,7 @@ def get_version(): exec(open(os.path.normpath('./apache_beam/version.py')).read(), global_names) return global_names['__version__'] + PACKAGE_NAME = 'apache-beam' PACKAGE_VERSION = get_version() PACKAGE_DESCRIPTION = 'Apache Beam SDK for Python' @@ -93,11 +94,26 @@ def get_version(): except ImportError: cythonize = lambda *args, **kwargs: [] - -REQUIRED_PACKAGES = [ +if sys.version_info[0] >= 3: + REQUIRED_PACKAGES = [ + 'avro-python3>=1.8.0,<2.0.0', + 'crcmod>=1.7,<2.0', + 'dill==0.2.7.1', + 'grpcio>=1.0,<2.0', + 'httplib2>=0.8,<0.10', + 'mock>=1.0.1,<3.0.0', + 'oauth2client>=2.0.1,<4.0.0', + 'protobuf>=3.2.0,<=3.3.0', + 'pyyaml>=3.12,<4.0.0', + 'typing>=3.6.0,<3.7.0', + 'future>=0.16.0', + 'six>=1.9', + ] +else: + REQUIRED_PACKAGES = [ 'avro>=1.8.1,<2.0.0', 'crcmod>=1.7,<2.0', - 'dill==0.2.6', + 'dill==0.2.7.1', 'grpcio>=1.0,<2.0', 'httplib2>=0.8,<0.10', 'mock>=1.0.1,<3.0.0', @@ -105,7 +121,9 @@ def get_version(): 'protobuf>=3.2.0,<=3.3.0', 'pyyaml>=3.12,<4.0.0', 'typing>=3.6.0,<3.7.0', - ] + 'future>=0.16.0', + 'six>=1.9', + ] REQUIRED_SETUP_PACKAGES = [ 'nose>=1.0', @@ -133,6 +151,7 @@ def generate_protos_first(original_cmd): # See https://issues.apache.org/jira/browse/BEAM-2366 # pylint: disable=wrong-import-position import gen_protos + class cmd(original_cmd, object): def run(self): gen_protos.generate_proto_files()