You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/08/06 03:15:01 UTC

[jira] [Work logged] (BEAM-2784) Fix issues from automated conversion to allow Python 2 functionality

     [ https://issues.apache.org/jira/browse/BEAM-2784?focusedWorklogId=131297&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-131297 ]

ASF GitHub Bot logged work on BEAM-2784:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 06/Aug/18 03:14
            Start Date: 06/Aug/18 03:14
    Worklog Time Spent: 10m 
      Work Description: holdenk closed pull request #3772: [WIP][BEAM-2784] Run python 2 to 3 migration and fix resulting Python 2 errors
URL: https://github.com/apache/beam/pull/3772
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/python/apache_beam/__init__.py b/sdks/python/apache_beam/__init__.py
index 791ebb7a342..eec352d60b1 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 2670250c36b..230cefb8d44 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 e204369b310..461598f852f 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 c89e81028b8..61385eee660 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 8b0353d922e..fdde00c6cb4 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 a13334a2c26..d060117a147 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 fc952cf4e55..5977624883f 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 eaf1aec9e60..09ca3041c29 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 16b1b4d6b4b..433d33f9884 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 97aa39ca094..7032e081658 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 1ab55d90f98..b7f3cd5fc50 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 5f984556447..39e03463aa1 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 e6108b68aee..b2a58a16700 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 3894bb58817..1be98f34115 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 2b6aa7a5129..0ce14f17e3d 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 ab3397cfe33..271a84ee7e6 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 e2c84d68d3d..0a1eeb33048 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 7e3c4cd35a2..d7b5895e7be 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 f1cbb0a24d5..94399da3249 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 9dd8b05a5c5..e6aeddb0ec0 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 9c30127d741..39fa72cb46d 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 59903d98546..e2880213bf3 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 61e3fd1a8d0..1f69e40e566 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 91c75aa5792..130978a9d85 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 0db5431e31c..1d521bec2e4 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 589e47c8305..cbf5f3d1e58 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 a88ff827766..1c97e43680e 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 b6f88255887..667951ff0d2 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 9a9ad7865c2..0813bd01b52 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 ced8a44af13..bdf65be9f9c 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 9911a6716bd..380b3cd2981 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 964b35b3f08..2be7abaf21f 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 1ca49c57d59..7b40353c0a8 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 05ee3c53f48..96128490804 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 45dcabaf853..2d12db5050f 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 f97b0f2d9c9..0ed02371533 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 988d3c9d25e..9bad505cea3 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 ee1fb7765d4..26c9f60b256 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 aee69d23ff6..282773293f7 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 7aaccb4ac1c..88d83fe2a6e 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 7161cff1c25..63ebd0f3721 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 fd49f938742..ff70caf7cba 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 9c0d04b816a..16d642e01d8 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 fb630ba465f..ed38b5d8a4e 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 9acdd907347..1a9af40cb4e 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"<TAB>"email_1,email_2"<TAB>"phone"<TAB>"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 2316c6611c0..5bc899144ed 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 27b81208fcd..9b6aefdf642 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 9183d0dfea1..15d762a5687 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 7696d778932..50151f38cc9 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 bd57847c67f..c10eb2dbde4 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 34dedb2b819..54a779745ff 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 c0ffd356364..9ec9d1bc956 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 8d2e73e28f6..dc8e8482c80 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 76b0a221df7..f0b30109c5d 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 5ee7b780398..b361ce39289 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 9834ba53111..8ffa7733b78 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 9f32092f416..ffe7d52c65e 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 167b173b1f8..ad9b760b0b8 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 b1fd63f048e..723431e7578 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 c7bb320253a..c74819598e1 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 e049a713423..146abaed656 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 05062d282da..a88aeaa1861 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 dbbeafcc1e7..79811c0eff1 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 9a2e3977a70..374307ac09b 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 cb14c654217..d0dbf9d13c2 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 969f4405085..a42fff7833f 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 56c4ccabcf7..7364f5445bf 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 4a8f5193dee..8df0ef3b7be 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 eb99d084901..6ebe756f8b7 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 7c8ddb4072e..d43e98dcb4c 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 649693076c5..52702be204c 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 afb340d7608..8c38be718ac 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 5804d0048af..96690179b17 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 607393d3a55..70ace6804e3 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 e03968628d5..0a18f646d02 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 9a6f013ff47..4627c2270d9 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 33d67bf7c53..e817dd784c8 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 bfd06ace640..9a3d0052289 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 7d94f24ca85..d547f61237c 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 93b91ad7cfa..c268dcae29c 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 3cfba93d2e8..05450cac1ba 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 72c4c8cc0c1..8c025b190cd 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 aa3780558d7..b9f546bccfe 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 5cde25540da..fe5b41e4c64 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 a8b1bb11231..43b6d0c5003 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 d5674f9cbf1..1c605d12e8f 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 52f25facd05..80ca2277c74 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 5670a241ba8..4579d0bc68d 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 ce8b5e6e424..0ac72b59ad4 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 923fc7dd4b6..a17cd50e7cb 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 b18a2875aa0..9c3b9dc98f9 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 06a82272900..714b689473d 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 201a1830b87..b32fc5638f7 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 404542863f3..4f3bae3d6ab 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 1b46d917f14..72b453dab19 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 dc9e5e69255..d1d226881ad 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 7d1f3553f4f..8e1673a8f23 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 0dcc3c39ab5..ee67bae3058 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 270750a2521..702c43b4955 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 db75fe38baa..deef1175620 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 b08ac494846..c7fb7e4aa97 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 04cf5b7c6f4..8d97d60377e 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 1339b911efc..66a3b510fe1 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 762d6547891..199e76f6fe0 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 bea97085083..a96977603f9 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 00522c9fd3f..3a271b382c9 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 10d401b8d2a..76df095c0c3 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 9708df7c753..1f58fa38028 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 b29ca5a9713..09e3c9bf133 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 d7eb932cd65..01838a11afc 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('<Q', len(value))
-    file_handle.write('{}{}{}{}'.format(
+    encoded_length = struct.pack(b'<Q', len(value))
+    pack_length = struct.pack(b'<I', cls._masked_crc32c(encoded_length))
+    pack_value = struct.pack(b'<I', cls._masked_crc32c(value))
+    output_str = b'{}{}{}{}'.format(
         encoded_length,
-        struct.pack('<I', cls._masked_crc32c(encoded_length)),  #
+        pack_length,
         value,
-        struct.pack('<I', cls._masked_crc32c(value))))
+        pack_value).decode("latin-1")
+    file_handle.write(output_str)
 
   @classmethod
   def read_record(cls, file_handle):
@@ -117,6 +126,7 @@ def read_record(cls, file_handle):
     if len(buf) != buf_length_expected:
       raise ValueError('Not a valid TFRecord. Fewer than %d bytes: %s' %
                        (buf_length_expected, buf.encode('hex')))
+    buf = buf.decode("latin-1").encode("latin-1")
     length, length_mask_expected = struct.unpack('<QI', buf)
     length_mask_actual = cls._masked_crc32c(buf[:8])
     if length_mask_actual != length_mask_expected:
diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py
index 3c70ade3dbe..b4835a44f65 100644
--- a/sdks/python/apache_beam/io/tfrecordio_test.py
+++ b/sdks/python/apache_beam/io/tfrecordio_test.py
@@ -16,9 +16,9 @@
 #
 
 import binascii
-import cStringIO
 import glob
 import gzip
+import io
 import logging
 import os
 import pickle
@@ -26,19 +26,21 @@
 import shutil
 import tempfile
 import unittest
+from builtins import chr, range
+
+import crcmod
+from future import standard_library
 
 import apache_beam as beam
 from apache_beam import coders
 from apache_beam.io.filesystem import CompressionTypes
-from apache_beam.io.tfrecordio import _TFRecordSink
-from apache_beam.io.tfrecordio import _TFRecordSource
-from apache_beam.io.tfrecordio import _TFRecordUtil
-from apache_beam.io.tfrecordio import ReadFromTFRecord
-from apache_beam.io.tfrecordio import WriteToTFRecord
+from apache_beam.io.tfrecordio import (ReadFromTFRecord, WriteToTFRecord,
+                                       _TFRecordSink, _TFRecordSource,
+                                       _TFRecordUtil)
 from apache_beam.testing.test_pipeline import TestPipeline
-from apache_beam.testing.util import assert_that
-from apache_beam.testing.util import equal_to
-import crcmod
+from apache_beam.testing.util import assert_that, equal_to
+
+standard_library.install_aliases()
 
 
 try:
@@ -68,14 +70,14 @@ def setUp(self):
     self.record = binascii.a2b_base64(FOO_RECORD_BASE64)
 
   def _as_file_handle(self, contents):
-    result = cStringIO.StringIO()
-    result.write(contents)
-    result.reset()
+    result = io.StringIO()
+    result.write(contents.decode("latin-1"))
+    result.seek(0)
     return result
 
   def _increment_value_at_index(self, value, index):
     l = list(value)
-    l[index] = chr(ord(l[index]) + 1)
+    l[index] = chr(ord(l[index]) + 1).encode("latin-1")
     return ''.join(l)
 
   def _test_error(self, record, error_text):
@@ -110,7 +112,7 @@ def test_masked_crc32c_crcmod(self):
             '\x03\x00\x00\x00\x00\x00\x00\x00', crc32c_fn=crc32c_fn))
 
   def test_write_record(self):
-    file_handle = cStringIO.StringIO()
+    file_handle = io.StringIO()
     _TFRecordUtil.write_record(file_handle, 'foo')
     self.assertEqual(self.record, file_handle.getvalue())
 
@@ -131,9 +133,9 @@ def test_read_record_invalid_data_mask(self):
 
   def test_compatibility_read_write(self):
     for record in ['', 'blah', 'another blah']:
-      file_handle = cStringIO.StringIO()
+      file_handle = io.StringIO()
       _TFRecordUtil.write_record(file_handle, record)
-      file_handle.reset()
+      file_handle.seek(0)
       actual = _TFRecordUtil.read_record(file_handle)
       self.assertEqual(record, actual)
 
@@ -322,9 +324,9 @@ def test_process_gzip_auto(self):
 class TestEnd2EndWriteAndRead(_TestCaseWithTempDirCleanUp):
 
   def create_inputs(self):
-    input_array = [[random.random() - 0.5 for _ in xrange(15)]
-                   for _ in xrange(12)]
-    memfile = cStringIO.StringIO()
+    input_array = [[random.random() - 0.5 for _ in range(15)]
+                   for _ in range(12)]
+    memfile = io.BytesIO()
     pickle.dump(input_array, memfile)
     return memfile.getvalue()
 
@@ -374,7 +376,7 @@ def test_end2end_example_proto(self):
     file_path_prefix = os.path.join(self._new_tempdir(), 'result')
 
     example = tf.train.Example()
-    example.features.feature['int'].int64_list.value.extend(range(3))
+    example.features.feature['int'].int64_list.value.extend(list(range(3)))
     example.features.feature['bytes'].bytes_list.value.extend(
         [b'foo', b'bar'])
 
diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py
index ba840f752b1..c910308eb7a 100644
--- a/sdks/python/apache_beam/metrics/cells.py
+++ b/sdks/python/apache_beam/metrics/cells.py
@@ -23,11 +23,14 @@
 Cells depend on a 'dirty-bit' in the CellCommitState class that tracks whether
 a cell's updates have been committed.
 """
+from __future__ import division
 
 import threading
+from builtins import object
 
-from apache_beam.metrics.metricbase import Counter
-from apache_beam.metrics.metricbase import Distribution
+from past.utils import old_div
+
+from apache_beam.metrics.metricbase import Counter, Distribution
 
 __all__ = ['DistributionResult']
 
@@ -234,7 +237,7 @@ def mean(self):
     """
     if self.data.count == 0:
       return None
-    return float(self.data.sum)/self.data.count
+    return old_div(float(self.data.sum), self.data.count)
 
 
 class DistributionData(object):
diff --git a/sdks/python/apache_beam/metrics/cells_test.py b/sdks/python/apache_beam/metrics/cells_test.py
index a4c8a43705f..7450922512f 100644
--- a/sdks/python/apache_beam/metrics/cells_test.py
+++ b/sdks/python/apache_beam/metrics/cells_test.py
@@ -17,11 +17,10 @@
 
 import threading
 import unittest
+from builtins import range
 
-from apache_beam.metrics.cells import CounterCell
-from apache_beam.metrics.cells import DistributionCell
-from apache_beam.metrics.cells import DistributionData
-from apache_beam.metrics.cells import CellCommitState
+from apache_beam.metrics.cells import (CellCommitState, CounterCell,
+                                       DistributionCell, DistributionData)
 
 
 class TestCounterCell(unittest.TestCase):
diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py
index 675e49c1234..31a5eab9aa6 100644
--- a/sdks/python/apache_beam/metrics/execution.py
+++ b/sdks/python/apache_beam/metrics/execution.py
@@ -29,8 +29,9 @@
 - MetricsContainer - Holds the metrics of a single step and a single
     unit-of-commit (bundle).
 """
-from collections import defaultdict
 import threading
+from builtins import object
+from collections import defaultdict
 
 from apache_beam.metrics.cells import CounterCell, DistributionCell
 
@@ -166,11 +167,11 @@ def _get_updates(self, filter=None):
     if filter is None:
       filter = lambda v: True
     counters = {MetricKey(self.step_name, k): v.get_cumulative()
-                for k, v in self.counters.items()
+                for k, v in list(self.counters.items())
                 if filter(v)}
 
     distributions = {MetricKey(self.step_name, k): v.get_cumulative()
-                     for k, v in self.distributions.items()
+                     for k, v in list(self.distributions.items())
                      if filter(v)}
 
     return MetricUpdates(counters, distributions)
diff --git a/sdks/python/apache_beam/metrics/execution_test.py b/sdks/python/apache_beam/metrics/execution_test.py
index 54569c10e99..ce6ca73d68d 100644
--- a/sdks/python/apache_beam/metrics/execution_test.py
+++ b/sdks/python/apache_beam/metrics/execution_test.py
@@ -16,12 +16,12 @@
 #
 
 import unittest
+from builtins import range
 
 from apache_beam.metrics.cells import CellCommitState
-from apache_beam.metrics.execution import MetricsContainer
-from apache_beam.metrics.execution import ScopedMetricsContainer
-from apache_beam.metrics.execution import MetricsEnvironment
-from apache_beam.metrics.execution import MetricKey
+from apache_beam.metrics.execution import (MetricKey, MetricsContainer,
+                                           MetricsEnvironment,
+                                           ScopedMetricsContainer)
 from apache_beam.metrics.metric import Metrics
 from apache_beam.metrics.metricbase import MetricName
 
@@ -29,9 +29,9 @@
 class TestMetricsContainer(unittest.TestCase):
   def test_create_new_counter(self):
     mc = MetricsContainer('astep')
-    self.assertFalse(mc.counters.has_key(MetricName('namespace', 'name')))
+    self.assertFalse(MetricName('namespace', 'name') in mc.counters)
     mc.get_counter(MetricName('namespace', 'name'))
-    self.assertTrue(mc.counters.has_key(MetricName('namespace', 'name')))
+    self.assertTrue(MetricName('namespace', 'name') in mc.counters)
 
   def test_scoped_container(self):
     c1 = MetricsContainer('mystep')
@@ -46,14 +46,14 @@ def test_scoped_container(self):
         counter = Metrics.counter('ns', 'name')
         counter.inc(3)
         self.assertEqual(
-            c2.get_cumulative().counters.items(),
+            list(c2.get_cumulative().counters.items()),
             [(MetricKey('myinternalstep', MetricName('ns', 'name')), 3)])
 
       self.assertEqual(c1, MetricsEnvironment.current_container())
       counter = Metrics.counter('ns', 'name')
       counter.inc(4)
       self.assertEqual(
-          c1.get_cumulative().counters.items(),
+          list(c1.get_cumulative().counters.items()),
           [(MetricKey('mystep', MetricName('ns', 'name')), 6)])
 
   def test_add_to_counter(self):
@@ -94,13 +94,13 @@ def test_get_cumulative_or_updates(self):
     self.assertEqual(len(logical.counters), 5)
     self.assertEqual(len(logical.distributions), 5)
     self.assertEqual(set(dirty_values),
-                     set([v for _, v in logical.counters.items()]))
+                     set([v for _, v in list(logical.counters.items())]))
     # Retrieve ALL updates.
     cumulative = mc.get_cumulative()
     self.assertEqual(len(cumulative.counters), 10)
     self.assertEqual(len(cumulative.distributions), 10)
     self.assertEqual(set(dirty_values + clean_values),
-                     set([v for _, v in cumulative.counters.items()]))
+                     set([v for _, v in list(cumulative.counters.items())]))
 
 
 class TestMetricsEnvironment(unittest.TestCase):
@@ -115,11 +115,11 @@ def test_uses_right_container(self):
     MetricsEnvironment.unset_current_container()
 
     self.assertEqual(
-        c1.get_cumulative().counters.items(),
+        list(c1.get_cumulative().counters.items()),
         [(MetricKey('step1', MetricName('ns', 'name')), 1)])
 
     self.assertEqual(
-        c2.get_cumulative().counters.items(),
+        list(c2.get_cumulative().counters.items()),
         [(MetricKey('step2', MetricName('ns', 'name')), 3)])
 
   def test_no_container(self):
diff --git a/sdks/python/apache_beam/metrics/metric.py b/sdks/python/apache_beam/metrics/metric.py
index f99c0c415d5..a156f60440d 100644
--- a/sdks/python/apache_beam/metrics/metric.py
+++ b/sdks/python/apache_beam/metrics/metric.py
@@ -25,10 +25,10 @@
     metric objects such as counters, distributions, etc.
 """
 import inspect
+from builtins import object
 
 from apache_beam.metrics.execution import MetricsEnvironment
-from apache_beam.metrics.metricbase import Counter, Distribution
-from apache_beam.metrics.metricbase import MetricName
+from apache_beam.metrics.metricbase import Counter, Distribution, MetricName
 
 __all__ = ['Metrics', 'MetricsFilter']
 
diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py
index 56b74680a97..beae78e2e68 100644
--- a/sdks/python/apache_beam/metrics/metric_test.py
+++ b/sdks/python/apache_beam/metrics/metric_test.py
@@ -16,14 +16,12 @@
 #
 
 import unittest
+from builtins import object
 
 from apache_beam.metrics.cells import DistributionData
-from apache_beam.metrics.execution import MetricKey
-from apache_beam.metrics.execution import MetricsContainer
-from apache_beam.metrics.execution import MetricsEnvironment
-from apache_beam.metrics.metric import Metrics
-from apache_beam.metrics.metric import MetricsFilter
-from apache_beam.metrics.metric import MetricResults
+from apache_beam.metrics.execution import (MetricKey, MetricsContainer,
+                                           MetricsEnvironment)
+from apache_beam.metrics.metric import MetricResults, Metrics, MetricsFilter
 from apache_beam.metrics.metricbase import MetricName
 
 
diff --git a/sdks/python/apache_beam/metrics/metricbase.py b/sdks/python/apache_beam/metrics/metricbase.py
index 699f29cb58a..aaa2592919a 100644
--- a/sdks/python/apache_beam/metrics/metricbase.py
+++ b/sdks/python/apache_beam/metrics/metricbase.py
@@ -30,6 +30,8 @@
 - MetricName - Namespace and name used to refer to a Metric.
 """
 
+from builtins import object
+
 __all__ = ['Metric', 'Counter', 'Distribution', 'MetricName']
 
 
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index db65b3c6583..1eaf9238cc9 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -18,12 +18,12 @@
 """Pipeline options obtained from command line parsing."""
 
 import argparse
+from builtins import object
 
+from apache_beam.options.value_provider import (RuntimeValueProvider,
+                                                StaticValueProvider,
+                                                ValueProvider)
 from apache_beam.transforms.display import HasDisplayData
-from apache_beam.options.value_provider import StaticValueProvider
-from apache_beam.options.value_provider import RuntimeValueProvider
-from apache_beam.options.value_provider import ValueProvider
-
 
 __all__ = [
     'PipelineOptions',
@@ -51,7 +51,7 @@ def _static_value_provider_of(value_type):
 
   """
   def _f(value):
-    _f.func_name = value_type.__name__
+    _f.__name__ = value_type.__name__
     return StaticValueProvider(value_type, value)
   return _f
 
@@ -175,7 +175,7 @@ def from_dictionary(cls, options):
       A PipelineOptions object representing the given arguments.
     """
     flags = []
-    for k, v in options.iteritems():
+    for k, v in options.items():
       if isinstance(v, bool):
         if v:
           flags.append('--%s' % k)
@@ -204,13 +204,13 @@ def get_all_options(self, drop_default=False):
     parser = _BeamArgumentParser()
     for cls in PipelineOptions.__subclasses__():
       subset[str(cls)] = cls
-    for cls in subset.values():
+    for cls in list(subset.values()):
       cls._add_argparse_args(parser)  # pylint: disable=protected-access
     known_args, _ = parser.parse_known_args(self._flags)
     result = vars(known_args)
 
     # Apply the overrides if any
-    for k in result.keys():
+    for k in list(result.keys()):
       if k in self._all_options:
         result[k] = self._all_options[k]
       if (drop_default and
@@ -233,7 +233,7 @@ def _visible_option_list(self):
                   for option in dir(self._visible_options) if option[0] != '_')
 
   def __dir__(self):
-    return sorted(dir(type(self)) + self.__dict__.keys() +
+    return sorted(dir(type(self)) + list(self.__dict__.keys()) +
                   self._visible_option_list())
 
   def __getattr__(self, name):
@@ -668,6 +668,6 @@ def wrapper(*args, **kwargs):
   @classmethod
   def augment_options(cls, options):
     for override in cls.overrides:
-      for name, value in override.items():
+      for name, value in list(override.items()):
         setattr(options, name, value)
     return options
diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py
index f4dd4d92b78..cc42ae9f791 100644
--- a/sdks/python/apache_beam/options/pipeline_options_test.py
+++ b/sdks/python/apache_beam/options/pipeline_options_test.py
@@ -21,11 +21,12 @@
 import unittest
 
 import hamcrest as hc
+
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.value_provider import (RuntimeValueProvider,
+                                                StaticValueProvider)
 from apache_beam.transforms.display import DisplayData
 from apache_beam.transforms.display_test import DisplayDataItemMatcher
-from apache_beam.options.pipeline_options import PipelineOptions
-from apache_beam.options.value_provider import StaticValueProvider
-from apache_beam.options.value_provider import RuntimeValueProvider
 
 
 class PipelineOptionsTest(unittest.TestCase):
diff --git a/sdks/python/apache_beam/options/pipeline_options_validator.py b/sdks/python/apache_beam/options/pipeline_options_validator.py
index 24d2e55eadb..e92fadc986a 100644
--- a/sdks/python/apache_beam/options/pipeline_options_validator.py
+++ b/sdks/python/apache_beam/options/pipeline_options_validator.py
@@ -20,15 +20,14 @@
 For internal use only; no backwards-compatibility guarantees.
 """
 import re
+from builtins import object
 
 from apache_beam.internal import pickler
-from apache_beam.options.pipeline_options import DebugOptions
-from apache_beam.options.pipeline_options import GoogleCloudOptions
-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.options.pipeline_options import TypeOptions
-from apache_beam.options.pipeline_options import WorkerOptions
+from apache_beam.options.pipeline_options import (DebugOptions,
+                                                  GoogleCloudOptions,
+                                                  SetupOptions,
+                                                  StandardOptions, TestOptions,
+                                                  TypeOptions, WorkerOptions)
 
 
 class PipelineOptionsValidator(object):
diff --git a/sdks/python/apache_beam/options/pipeline_options_validator_test.py b/sdks/python/apache_beam/options/pipeline_options_validator_test.py
index 97834cc4821..f4635ac6759 100644
--- a/sdks/python/apache_beam/options/pipeline_options_validator_test.py
+++ b/sdks/python/apache_beam/options/pipeline_options_validator_test.py
@@ -19,11 +19,14 @@
 
 import logging
 import unittest
+from builtins import object
+
+from hamcrest.core.base_matcher import BaseMatcher
 
 from apache_beam.internal import pickler
 from apache_beam.options.pipeline_options import PipelineOptions
-from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator
-from hamcrest.core.base_matcher import BaseMatcher
+from apache_beam.options.pipeline_options_validator import \
+    PipelineOptionsValidator
 
 
 # Mock runners to use for validations.
diff --git a/sdks/python/apache_beam/options/value_provider.py b/sdks/python/apache_beam/options/value_provider.py
index f7630bd8d32..e0052bd2b74 100644
--- a/sdks/python/apache_beam/options/value_provider.py
+++ b/sdks/python/apache_beam/options/value_provider.py
@@ -19,11 +19,11 @@
 and dynamically provided values.
 """
 
+from builtins import object
 from functools import wraps
 
 from apache_beam import error
 
-
 __all__ = [
     'ValueProvider',
     'StaticValueProvider',
diff --git a/sdks/python/apache_beam/options/value_provider_test.py b/sdks/python/apache_beam/options/value_provider_test.py
index 17e9590d2a3..60d10a17135 100644
--- a/sdks/python/apache_beam/options/value_provider_test.py
+++ b/sdks/python/apache_beam/options/value_provider_test.py
@@ -20,8 +20,8 @@
 import unittest
 
 from apache_beam.options.pipeline_options import PipelineOptions
-from apache_beam.options.value_provider import RuntimeValueProvider
-from apache_beam.options.value_provider import StaticValueProvider
+from apache_beam.options.value_provider import (RuntimeValueProvider,
+                                                StaticValueProvider)
 
 
 # TODO(BEAM-1319): Require unique names only within a test.
diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py
index 1ade6c067c5..372156eb493 100644
--- a/sdks/python/apache_beam/pipeline.py
+++ b/sdks/python/apache_beam/pipeline.py
@@ -51,23 +51,28 @@
 import logging
 import os
 import shutil
+import sys
 import tempfile
+from builtins import object
+
+from future.utils import with_metaclass
 
 from apache_beam import pvalue
 from apache_beam.internal import pickler
+from apache_beam.options.pipeline_options import (PipelineOptions,
+                                                  SetupOptions,
+                                                  StandardOptions, TypeOptions)
+from apache_beam.options.pipeline_options_validator import \
+    PipelineOptionsValidator
 from apache_beam.pvalue import PCollection
-from apache_beam.runners import create_runner
-from apache_beam.runners import PipelineRunner
+from apache_beam.runners import PipelineRunner, create_runner
 from apache_beam.transforms import ptransform
-from apache_beam.typehints import typehints
-from apache_beam.typehints import TypeCheckError
-from apache_beam.options.pipeline_options import PipelineOptions
-from apache_beam.options.pipeline_options import SetupOptions
-from apache_beam.options.pipeline_options import StandardOptions
-from apache_beam.options.pipeline_options import TypeOptions
-from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator
-from apache_beam.utils.annotations import deprecated
+from apache_beam.typehints import TypeCheckError, typehints
 from apache_beam.utils import urns
+from apache_beam.utils.annotations import deprecated
+
+if sys.version_info[0] >= 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 b3ac100780f..dee80730cd5 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 34a483e7bb9..73a8e76baac 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 84535692806..7872453dcfa 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 62a6955f6ce..e0f628c71ee 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 24916fd4927..f7a931ad97a 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 dd3cbe1156a..5c00fce3322 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 813759e1253..b12b57f3fd9 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 8e708e6fde2..46433d965a7 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 a1f9301b19b..84c48d77aa3 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 407ffcf2ad7..e7cced7b4f3 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 61d02730ab5..977a4c03c03 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 e81329fca80..6f8e3e755ca 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 4dda47ad9e0..35c6ed24298 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 3163c9b2512..22b13f511ef 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 f5dd2bd3573..d5e61241439 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 e555b69f15a..83fecb18910 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 be6722427b6..fedc211ca2e 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 2f2316f6f1d..5ee3c221056 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 3d8c24f5651..0fa47f9793b 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 8c6c8d6d529..533a160616c 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 7927219ab5e..88afe8a2ee6 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 96e6a66caab..12ad72eba38 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 0182b4c9e9c..5a35dc675b9 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 84d52f79948..e019af27fb1 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 97d1ee8bfb6..0263904a734 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 9d234876035..5500b083176 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 256b91f3e9a..a9b3120d1dd 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 2deb7dace02..22dadff3923 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 54c407c1c86..f207c612695 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 3e08b522d54..1571b33646f 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 374cd4ea69d..26b0701bd02 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 cb2ace29f0e..b35859c07d7 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 10f7b294c13..1f745ae0d97 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 935998d27de..067593c6c5c 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 247ce1f0e19..b94874cf27c 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 bae25a4d07b..8e56f9f59f6 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 4d88a1189f2..49f9cbf448f 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 84c727fb4eb..80b5e451f2d 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 42d7f5df14a..a98a5588c53 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 7c0c06fe111..146810cc8d5 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 ba219547e6c..ef7660518bd 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 ddfc4ccc43f..39bcb3de9d0 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 b7ba15a502a..0d9861566c6 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 43ee27bc298..8d8ae6f022c 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 fa80b1c9555..1b3b1b4f69f 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 16c888c63c6..d5ea7b26caf 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 e713041d7c8..0c3c76d038b 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 360468a8687..be6a15c2572 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 b8f635210d2..91b4e3156dc 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 2256bb5556f..9702493a8a7 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 cf3f6929282..b5426625fc6 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 2bb15fa7ee4..f48cc6fa443 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 74561b81593..def6be7f9c7 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 bdafbeaf44a..06147eaab64 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 c4f945bf2b9..fd655921f9a 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 6a236802b9a..e77b3ed969d 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 dc72a5ff4fa..ffb2ca04bed 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 bdf9f4e71f5..9375489b6cb 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 d243bbe4e6e..e73f39ddc37 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 88ace8c5ae8..aa0fe342e17 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 663cdecdab0..8569a4fe8ac 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 883343acdab..c421e259398 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 15e0a042008..da8adc9cc25 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 83802423dbc..ea0403371e6 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 747d64c7409..c642c65a7b7 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 7989fb2eee9..ef2eb10debf 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 b7ca141f059..caafe4f245d 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 26ca03d1375..de7188795b6 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 bee0bd361e2..093e1f175d2 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 959f25f3187..6b1d92c7791 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 875306f8082..d283efee6f6 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 cd2b5956fef..6f843975e51 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 d6f56d2d0cf..d259c32a2c5 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 55ad7f32982..794248ffd57 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 84aee212790..21fd72d194f 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 88a1feef75e..81679d9eb67 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 15f178645bc..d5578a6a29b 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 f6e08ca9c0c..94528c30793 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__ == '<lambda>':
       return '<lambda at %s:%s>' % (
-          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 efc597877b2..2ddf0dbbd48 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 <type 'str'> but got <type 'int'> 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 <type 'int'> but got <type 'str'> 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 <type 'int'> but got <type 'str'> 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 <type 'str'> but got <type 'int'> 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 <type 'str'> but got <type 'int'> 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 <type 'str'> but got <type 'int'> 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 <type 'int'> but got <type 'str'> 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 <type 'float'> but got <type 'int'> 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 <type 'int'>",
-                     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 <type 'int'>, "
@@ -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 <type 'str'>, "
@@ -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 <type 'int'>, "
-        "instead found 3.0, an instance of <type 'float'>.")
+        "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 <type 'int'>, "
@@ -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 <type 'int'>, "
@@ -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: <type 'int'> 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 <type 'str'>, got <type 'int'>",
-                     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 <type 'int'> 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 <type 'int'> 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 f10cb92ed5e..21fc919b72d 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 65006814185..a4fd5377561 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 c0f91981d7d..5ba47d17273 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 c1fbfc54a3b..8ed5fe5b743 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 1ae1f025032..a0db2678788 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 a7484acb2cc..2b3bdf11032 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 458fb747eb2..beb7ced000a 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 977a364ad88..13080dbef16 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 50f0debb0a7..dc4d333198f 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 3f5b4c9c479..c0fd9385238 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 26c584e380d..6a0e0371428 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 0ff2b3b5e58..faca02650a9 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 83f444c6c53..23f3fd2b87b 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 c7405963f16..db629bb2337 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 e7f451da11a..e993ae1866e 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 89a5f5c7e2c..1fbc01272e3 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 59d1e1c5714..4059941cc7c 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 98d399b82f3..b9376949aa7 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 f1b92e09626..3ffb42344a1 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: (<type 'int'>, <type "
-                     "'str'>, <type 'bool'>), have a length of 3.",
-                     e.exception.message)
+                     "'str'>, <type 'bool'>), 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: (<type 'float'>, <type "
                      "'int'>, <type 'bool'>), 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 <type 'int'>, instead found an "
                      "instance of <type 'str'>.",
-                     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 <type 'int'>, instead found an instance "
                      "of <type 'str'>.",
-                     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 <type 'int'>, instead found an instance "
                      "of <type 'str'>.",
-                     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 017dd6b81a4..46c8b58c7e2 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 32af8a991a7..94ed058b22d 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 08685aae275..d7c79962cf9 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 563b93c54c7..31b440948cc 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 920b621b1bb..2dd45f44dc5 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 a2c3f6ab1af..0e2cc3e83d8 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 d7693f3f783..79110474fe7 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 08223b3f1f7..541d95f6ad1 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 1b03c835cd7..5877159fa92 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 00000000000..d8809d12b7f
--- /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 b3e840ee284..c370ab4111d 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 33229361b36..04b19ed87a6 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 18959be8750..8e101fd8036 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 be2785432a1..dbf28a30a8b 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 a3d963d18d7..fa94d70b85d 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 662bd098e10..2967cfec174 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 ddc3dc7de77..3bd465ac6ca 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 c13da8e326c..10d5bddbf6d 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()


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 131297)
    Time Spent: 1h 20m  (was: 1h 10m)

> Fix issues from automated conversion to allow Python 2 functionality
> --------------------------------------------------------------------
>
>                 Key: BEAM-2784
>                 URL: https://issues.apache.org/jira/browse/BEAM-2784
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-core
>            Reporter: holdenk
>            Assignee: Holden Karau
>            Priority: Major
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> As part of BEAM-1251 we want to move to support a Python2/3 code base. To do this we can use futurize but futurize will break some Python2 elements. A good intermediate checkpoint is contiuing to support Python 2 after futurization from which we can build Python 3 support on top of.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)