diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 4a65c69f4e02..6fd9b169ed60 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -37,6 +37,9 @@ from builtins import chr from builtins import object +from past.builtins import long +from past.builtins import unicode + from apache_beam.coders import observable from apache_beam.utils import windowed_value from apache_beam.utils.timestamp import MAX_TIMESTAMP @@ -61,13 +64,6 @@ from .slow_stream import get_varint_size # pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports -try: # Python 2 - long # pylint: disable=long-builtin - unicode # pylint: disable=unicode-builtin -except NameError: # Python 3 - long = int - unicode = str - class CoderImpl(object): """For internal use only; no backwards-compatibility guarantees.""" diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 70237880cd21..cf4b9b5d520b 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -24,8 +24,8 @@ import base64 from builtins import object -import google.protobuf -from google.protobuf import wrappers_pb2 +import google.protobuf.wrappers_pb2 +from future.moves import pickle from apache_beam.coders import coder_impl from apache_beam.portability import common_urns @@ -33,12 +33,6 @@ from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.utils import proto_utils -# This is for py2/3 compatibility. cPickle was renamed pickle in python 3. -try: - import cPickle as pickle # Python 2 -except ImportError: - import pickle # Python 3 - # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports try: from .stream import get_varint_size @@ -277,7 +271,7 @@ def from_runner_api(cls, coder_proto, context): def to_runner_api_parameter(self, context): return ( python_urns.PICKLED_CODER, - wrappers_pb2.BytesValue(value=serialize_coder(self)), + google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), ()) @staticmethod diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py index e4efa2c7ffdf..7afb01585d44 100644 --- a/sdks/python/apache_beam/coders/typecoders.py +++ b/sdks/python/apache_beam/coders/typecoders.py @@ -67,14 +67,11 @@ def MakeXyzs(v): from builtins import object +from past.builtins import unicode + from apache_beam.coders import coders from apache_beam.typehints import typehints -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - __all__ = ['registry'] diff --git a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py index d09a46a14d52..b8c4213643ee 100644 --- a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py +++ b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py @@ -73,6 +73,7 @@ from google.cloud.proto.datastore.v1 import query_pb2 from googledatastore import helper as datastore_helper from googledatastore import PropertyFilter +from past.builtins import unicode import apache_beam as beam from apache_beam.io import ReadFromText @@ -83,11 +84,6 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class WordExtractingDoFn(beam.DoFn): """Parse each line of input text into words.""" diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 161a67b037fe..d854431e56a1 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -36,6 +36,8 @@ from builtins import object from builtins import range +from past.builtins import unicode + import apache_beam as beam from apache_beam.io import iobase from apache_beam.io.range_trackers import OffsetRangeTracker @@ -55,11 +57,6 @@ # pylint:disable=unused-variable # pylint:disable=wrong-import-order, wrong-import-position -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class SnippetUtils(object): from apache_beam.pipeline import PipelineVisitor diff --git a/sdks/python/apache_beam/examples/streaming_wordcount.py b/sdks/python/apache_beam/examples/streaming_wordcount.py index 388f9e39b5b2..8bc02fa9129a 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcount.py +++ b/sdks/python/apache_beam/examples/streaming_wordcount.py @@ -23,6 +23,8 @@ import argparse import logging +from past.builtins import unicode + import apache_beam as beam import apache_beam.transforms.window as window from apache_beam.examples.wordcount import WordExtractingDoFn @@ -30,11 +32,6 @@ from apache_beam.options.pipeline_options import SetupOptions from apache_beam.options.pipeline_options import StandardOptions -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - def run(argv=None): """Build and run the pipeline.""" diff --git a/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py index 419e247e8ef8..3f0d91213e3f 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py +++ b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py @@ -38,6 +38,8 @@ import logging import re +from past.builtins import unicode + import apache_beam as beam import apache_beam.transforms.window as window from apache_beam.examples.wordcount import WordExtractingDoFn @@ -48,11 +50,6 @@ from apache_beam.testing.util import equal_to_per_window from apache_beam.transforms.core import ParDo -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class PrintFn(beam.DoFn): """A DoFn that prints label, element, its window, and its timstamp. """ diff --git a/sdks/python/apache_beam/examples/windowed_wordcount.py b/sdks/python/apache_beam/examples/windowed_wordcount.py index b5c4fde7aa7c..1ea7abda64f9 100644 --- a/sdks/python/apache_beam/examples/windowed_wordcount.py +++ b/sdks/python/apache_beam/examples/windowed_wordcount.py @@ -26,14 +26,11 @@ import argparse import logging +from past.builtins import unicode + import apache_beam as beam import apache_beam.transforms.window as window -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - TABLE_SCHEMA = ('word:STRING, count:INTEGER, ' 'window_start:TIMESTAMP, window_end:TIMESTAMP') diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py index 3ba3b334188b..4492a737bd9a 100644 --- a/sdks/python/apache_beam/examples/wordcount.py +++ b/sdks/python/apache_beam/examples/wordcount.py @@ -23,6 +23,8 @@ import logging import re +from past.builtins import unicode + import apache_beam as beam from apache_beam.io import ReadFromText from apache_beam.io import WriteToText @@ -31,11 +33,6 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class WordExtractingDoFn(beam.DoFn): """Parse each line of input text into words.""" diff --git a/sdks/python/apache_beam/examples/wordcount_debugging.py b/sdks/python/apache_beam/examples/wordcount_debugging.py index 5d932d54ec60..23bb52b15691 100644 --- a/sdks/python/apache_beam/examples/wordcount_debugging.py +++ b/sdks/python/apache_beam/examples/wordcount_debugging.py @@ -45,6 +45,8 @@ import logging import re +from past.builtins import unicode + import apache_beam as beam from apache_beam.io import ReadFromText from apache_beam.io import WriteToText @@ -54,11 +56,6 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class FilterTextFn(beam.DoFn): """A DoFn that filters for a specific key based on a regular expression.""" diff --git a/sdks/python/apache_beam/examples/wordcount_fnapi.py b/sdks/python/apache_beam/examples/wordcount_fnapi.py index 00964c9ea46c..bf4998af15e3 100644 --- a/sdks/python/apache_beam/examples/wordcount_fnapi.py +++ b/sdks/python/apache_beam/examples/wordcount_fnapi.py @@ -28,6 +28,8 @@ import logging import re +from past.builtins import unicode + import apache_beam as beam from apache_beam.io import ReadFromText # TODO(BEAM-2887): Enable after the issue is fixed. @@ -38,11 +40,6 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class WordExtractingDoFn(beam.DoFn): """Parse each line of input text into words.""" diff --git a/sdks/python/apache_beam/examples/wordcount_minimal.py b/sdks/python/apache_beam/examples/wordcount_minimal.py index 1fbca3279641..ef66a38ed550 100644 --- a/sdks/python/apache_beam/examples/wordcount_minimal.py +++ b/sdks/python/apache_beam/examples/wordcount_minimal.py @@ -50,17 +50,14 @@ import logging import re +from past.builtins import unicode + 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 -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - def run(argv=None): """Main entry point; defines and runs the wordcount pipeline.""" diff --git a/sdks/python/apache_beam/internal/gcp/json_value.py b/sdks/python/apache_beam/internal/gcp/json_value.py index a4c6fb9d135a..ec85c288731e 100644 --- a/sdks/python/apache_beam/internal/gcp/json_value.py +++ b/sdks/python/apache_beam/internal/gcp/json_value.py @@ -19,6 +19,9 @@ from __future__ import absolute_import +from past.builtins import long +from past.builtins import unicode + from apache_beam.options.value_provider import ValueProvider # Protect against environments where apitools library is not available. @@ -29,12 +32,6 @@ extra_types = None # pylint: enable=wrong-import-order, wrong-import-position -try: # Python 2 - unicode # pylint: disable=unicode-builtin - long # pylint: disable=long-builtin -except NameError: # Python 3 - unicode = str - long = int _MAXINT64 = (1 << 63) - 1 _MININT64 = - (1 << 63) diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index c4a746f5289d..dc6726e6442a 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -28,6 +28,7 @@ from builtins import zip from future.utils import iteritems +from past.builtins import unicode from apache_beam.internal import util from apache_beam.io import iobase @@ -43,11 +44,6 @@ __all__ = ['FileBasedSink'] -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - class FileBasedSink(iobase.Sink): """A sink to a GCS or local files. diff --git a/sdks/python/apache_beam/io/filesystem.py b/sdks/python/apache_beam/io/filesystem.py index a9dafe6df703..e59b1f9f2f04 100644 --- a/sdks/python/apache_beam/io/filesystem.py +++ b/sdks/python/apache_beam/io/filesystem.py @@ -39,6 +39,8 @@ from future import standard_library from future.utils import with_metaclass +from past.builtins import long +from past.builtins import unicode from apache_beam.utils.plugin import BeamPlugin @@ -51,13 +53,6 @@ __all__ = ['CompressionTypes', 'CompressedFile', 'FileMetadata', 'FileSystem', 'MatchResult'] -try: - unicode # pylint: disable=unicode-builtin - long # pylint: disable=long-builtin -except NameError: - unicode = str - long = int - class CompressionTypes(object): """Enum-like class representing known compression types.""" diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py index 55ad5d1e33ed..91786b88c26b 100644 --- a/sdks/python/apache_beam/io/filesystems.py +++ b/sdks/python/apache_beam/io/filesystems.py @@ -22,6 +22,8 @@ import re from builtins import object +from past.builtins import unicode + from apache_beam.io.filesystem import BeamIOError from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.filesystem import FileSystem @@ -48,10 +50,6 @@ except ImportError: pass -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str # pylint: enable=wrong-import-position, unused-import diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index be5f3cb6cdac..ee39a6678dea 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -124,6 +124,7 @@ from future.utils import iteritems from future.utils import itervalues +from past.builtins import unicode from apache_beam import coders from apache_beam.internal.gcp import auth @@ -146,10 +147,6 @@ pass # pylint: enable=wrong-import-order, wrong-import-position -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str __all__ = [ 'TableRowJsonCoder', diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py index 2613b364b6f0..50439e198177 100644 --- a/sdks/python/apache_beam/io/range_trackers.py +++ b/sdks/python/apache_beam/io/range_trackers.py @@ -24,16 +24,13 @@ import math import threading +from past.builtins import long + from apache_beam.io import iobase __all__ = ['OffsetRangeTracker', 'LexicographicKeyRangeTracker', 'OrderedPositionRangeTracker', 'UnsplittableRangeTracker'] -try: - long # pylint: disable=long-builtin -except NameError: - long = int - class OffsetRangeTracker(iobase.RangeTracker): """A 'RangeTracker' for non-negative positions of type 'long'.""" diff --git a/sdks/python/apache_beam/io/range_trackers_test.py b/sdks/python/apache_beam/io/range_trackers_test.py index f0c7ca4ace85..475510e5d1fb 100644 --- a/sdks/python/apache_beam/io/range_trackers_test.py +++ b/sdks/python/apache_beam/io/range_trackers_test.py @@ -24,12 +24,9 @@ import math import unittest -from apache_beam.io import range_trackers +from past.builtins import long -try: - long # pylint: disable=long-builtin -except NameError: - long = int +from apache_beam.io import range_trackers class OffsetRangeTrackerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/vcfio.py b/sdks/python/apache_beam/io/vcfio.py index cd3b617e2c73..a5e606b26b1b 100644 --- a/sdks/python/apache_beam/io/vcfio.py +++ b/sdks/python/apache_beam/io/vcfio.py @@ -29,6 +29,7 @@ from collections import namedtuple from future.utils import iteritems +from past.builtins import unicode import vcf @@ -39,13 +40,6 @@ from apache_beam.io.textio import _TextSource as TextSource from apache_beam.transforms import PTransform -try: - unicode # pylint: disable=unicode-builtin - int # Python 2 -except NameError: - unicode = str - long = int # Python 3 - __all__ = ['ReadFromVcf', 'Variant', 'VariantCall', 'VariantInfo', 'MalformedVcfRecord'] diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 5668b9a782d0..62cefa12bc85 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -31,6 +31,8 @@ from builtins import hex from builtins import object +from past.builtins import unicode + from apache_beam import coders from apache_beam import typehints from apache_beam.internal import pickler @@ -38,11 +40,6 @@ from apache_beam.portability import python_urns from apache_beam.portability.api import beam_runner_api_pb2 -try: - unicode # pylint: disable=unicode-builtin -except NameError: - unicode = str - __all__ = [ 'PCollection', 'TaggedOutput', diff --git a/sdks/python/apache_beam/transforms/combiners.py b/sdks/python/apache_beam/transforms/combiners.py index b6f19c6c03eb..9b0c0e81e35e 100644 --- a/sdks/python/apache_beam/transforms/combiners.py +++ b/sdks/python/apache_beam/transforms/combiners.py @@ -22,6 +22,8 @@ import operator import random +from past.builtins import long + from apache_beam.transforms import core from apache_beam.transforms import cy_combiners from apache_beam.transforms import ptransform @@ -36,11 +38,6 @@ from apache_beam.typehints import with_input_types from apache_beam.typehints import with_output_types -try: - long # Python 2 -except NameError: - long = int # Python 3 - __all__ = [ 'Count', 'Mean', diff --git a/sdks/python/apache_beam/typehints/opcodes.py b/sdks/python/apache_beam/typehints/opcodes.py index a9874cfbda37..8bdf16b2191c 100644 --- a/sdks/python/apache_beam/typehints/opcodes.py +++ b/sdks/python/apache_beam/typehints/opcodes.py @@ -33,6 +33,8 @@ import types from functools import reduce +from past.builtins import unicode + from . import typehints from .trivial_inference import BoundMethod from .trivial_inference import Const @@ -45,11 +47,6 @@ from .typehints import Tuple from .typehints import Union -try: # Python 2 - unicode # pylint: disable=unicode-builtin -except NameError: # Python 3 - unicode = str - def pop_one(state, unused_arg): del state.stack[-1:] diff --git a/sdks/python/apache_beam/typehints/typecheck.py b/sdks/python/apache_beam/typehints/typecheck.py index 0754ea9ed479..72106a560e4a 100644 --- a/sdks/python/apache_beam/typehints/typecheck.py +++ b/sdks/python/apache_beam/typehints/typecheck.py @@ -27,6 +27,7 @@ import types from future.utils import raise_with_traceback +from past.builtins import unicode from apache_beam import pipeline from apache_beam.pvalue import TaggedOutput @@ -41,11 +42,6 @@ from apache_beam.typehints.typehints import SimpleTypeHintError from apache_beam.typehints.typehints import check_constraint -try: # Python 2 - unicode # pylint: disable=unicode-builtin -except NameError: # Python 3 - unicode = str - class AbstractDoFnWrapper(DoFn): """An abstract class to create wrapper around DoFn""" diff --git a/sdks/python/apache_beam/utils/timestamp.py b/sdks/python/apache_beam/utils/timestamp.py index e0c4093e7474..fe02ec273f04 100644 --- a/sdks/python/apache_beam/utils/timestamp.py +++ b/sdks/python/apache_beam/utils/timestamp.py @@ -29,11 +29,7 @@ from builtins import object import pytz - -try: # Python 2 - long # pylint: disable=long-builtin -except NameError: # Python 3 - long = int +from past.builtins import long @functools.total_ordering