Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 648f6ac

Browse files
authored
[BEAM-7372][BEAM-9372] Cleanup py2 and py35 codepath from runners (#14137)
1 parent f8da690 commit 648f6ac

26 files changed

Lines changed: 57 additions & 294 deletions

sdks/python/apache_beam/runners/dataflow/dataflow_runner.py

Lines changed: 4 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -28,16 +28,16 @@
2828
import base64
2929
import logging
3030
import os
31-
import subprocess
32-
import sys
3331
import threading
3432
import time
3533
import traceback
36-
import urllib
3734
from builtins import hex
3835
from collections import defaultdict
36+
from subprocess import DEVNULL
3937
from typing import TYPE_CHECKING
4038
from typing import List
39+
from urllib.parse import quote
40+
from urllib.parse import unquote_to_bytes
4141

4242
from future.utils import iteritems
4343

@@ -78,13 +78,6 @@
7878
if TYPE_CHECKING:
7979
from apache_beam.pipeline import PTransformOverride
8080

81-
if sys.version_info[0] > 2:
82-
unquote_to_bytes = urllib.parse.unquote_to_bytes
83-
quote = urllib.parse.quote
84-
else:
85-
unquote_to_bytes = urllib.unquote # pylint: disable=deprecated-urllib-function
86-
quote = urllib.quote # pylint: disable=deprecated-urllib-function
87-
8881
__all__ = ['DataflowRunner']
8982

9083
_LOGGER = logging.getLogger(__name__)
@@ -587,8 +580,7 @@ def run_pipeline(self, pipeline, options):
587580
# is set. Note that use_avro is only interpreted by the Dataflow runner
588581
# at job submission and is not interpreted by Dataflow service or workers,
589582
# which by default use avro library unless use_fastavro experiment is set.
590-
if sys.version_info[0] > 2 and (
591-
not debug_options.lookup_experiment('use_avro')):
583+
if not debug_options.lookup_experiment('use_avro'):
592584
debug_options.add_experiment('use_fastavro')
593585

594586
self.job = apiclient.Job(options, self.proto_pipeline)
@@ -1485,11 +1477,6 @@ def get_default_gcp_region(self):
14851477
return environment_region
14861478
try:
14871479
cmd = ['gcloud', 'config', 'get-value', 'compute/region']
1488-
# Use subprocess.DEVNULL in Python 3.3+.
1489-
if hasattr(subprocess, 'DEVNULL'):
1490-
DEVNULL = subprocess.DEVNULL
1491-
else:
1492-
DEVNULL = open(os.devnull, 'ab')
14931480
raw_output = processes.check_output(cmd, stderr=DEVNULL)
14941481
formatted_output = raw_output.decode('utf-8').strip()
14951482
if formatted_output:

sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -22,14 +22,11 @@
2222
from __future__ import absolute_import
2323

2424
import json
25-
import sys
2625
import unittest
2726
from builtins import object
2827
from builtins import range
2928
from datetime import datetime
3029

31-
# patches unittest.TestCase to be python3 compatible
32-
import future.tests.base # pylint: disable=unused-import
3330
import mock
3431

3532
import apache_beam as beam
@@ -504,8 +501,7 @@ def test_use_fastavro_experiment_is_added_on_py3_and_onwards(self):
504501
with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
505502
p | ptransform.Create([1]) # pylint: disable=expression-not-assigned
506503

507-
self.assertEqual(
508-
sys.version_info[0] > 2,
504+
self.assertTrue(
509505
remote_runner.job.options.view_as(DebugOptions).lookup_experiment(
510506
'use_fastavro', False))
511507

sdks/python/apache_beam/runners/dataflow/internal/apiclient.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1133,7 +1133,7 @@ def get_runner_harness_container_image():
11331133

11341134
def get_response_encoding():
11351135
"""Encoding to use to decode HTTP response from Google APIs."""
1136-
return None if sys.version_info[0] < 3 else 'utf8'
1136+
return 'utf8'
11371137

11381138

11391139
def _verify_interpreter_version_is_supported(pipeline_options):

sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py

Lines changed: 38 additions & 103 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,6 @@
2525
import sys
2626
import unittest
2727

28-
# patches unittest.TestCase to be python3 compatible
29-
import future.tests.base # pylint: disable=unused-import
3028
import mock
3129

3230
from apache_beam.metrics.cells import DistributionData
@@ -540,27 +538,14 @@ def test_pinned_worker_harness_image_tag_used_in_dev_sdk(self):
540538
pipeline_options,
541539
'2.0.0', #any environment version
542540
FAKE_PIPELINE_URL)
543-
if sys.version_info[0] == 2:
544-
self.assertEqual(
545-
env.proto.workerPools[0].workerHarnessContainerImage,
546-
(
547-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python-fnapi:' +
548-
names.BEAM_FNAPI_CONTAINER_VERSION))
549-
elif sys.version_info[0:2] == (3, 5):
550-
self.assertEqual(
551-
env.proto.workerPools[0].workerHarnessContainerImage,
552-
(
553-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3-fnapi:' +
554-
names.BEAM_FNAPI_CONTAINER_VERSION))
555-
else:
556-
self.assertEqual(
557-
env.proto.workerPools[0].workerHarnessContainerImage,
558-
(
559-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY +
560-
'/python%d%d-fnapi:%s' % (
561-
sys.version_info[0],
562-
sys.version_info[1],
563-
names.BEAM_FNAPI_CONTAINER_VERSION)))
541+
self.assertEqual(
542+
env.proto.workerPools[0].workerHarnessContainerImage,
543+
(
544+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d-fnapi:%s' %
545+
(
546+
sys.version_info[0],
547+
sys.version_info[1],
548+
names.BEAM_FNAPI_CONTAINER_VERSION)))
564549

565550
# batch, legacy pipeline.
566551
pipeline_options = PipelineOptions(
@@ -570,26 +555,13 @@ def test_pinned_worker_harness_image_tag_used_in_dev_sdk(self):
570555
pipeline_options,
571556
'2.0.0', #any environment version
572557
FAKE_PIPELINE_URL)
573-
if sys.version_info[0] == 2:
574-
self.assertEqual(
575-
env.proto.workerPools[0].workerHarnessContainerImage,
576-
(
577-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python:' +
578-
names.BEAM_CONTAINER_VERSION))
579-
elif sys.version_info[0:2] == (3, 5):
580-
self.assertEqual(
581-
env.proto.workerPools[0].workerHarnessContainerImage,
582-
(
583-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3:' +
584-
names.BEAM_CONTAINER_VERSION))
585-
else:
586-
self.assertEqual(
587-
env.proto.workerPools[0].workerHarnessContainerImage,
588-
(
589-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:%s' % (
590-
sys.version_info[0],
591-
sys.version_info[1],
592-
names.BEAM_CONTAINER_VERSION)))
558+
self.assertEqual(
559+
env.proto.workerPools[0].workerHarnessContainerImage,
560+
(
561+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:%s' % (
562+
sys.version_info[0],
563+
sys.version_info[1],
564+
names.BEAM_CONTAINER_VERSION)))
593565

594566
@mock.patch(
595567
'apache_beam.runners.dataflow.internal.apiclient.'
@@ -604,21 +576,12 @@ def test_worker_harness_image_tag_matches_released_sdk_version(self):
604576
pipeline_options,
605577
'2.0.0', #any environment version
606578
FAKE_PIPELINE_URL)
607-
if sys.version_info[0] == 2:
608-
self.assertEqual(
609-
env.proto.workerPools[0].workerHarnessContainerImage,
610-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python-fnapi:2.2.0'))
611-
elif sys.version_info[0:2] == (3, 5):
612-
self.assertEqual(
613-
env.proto.workerPools[0].workerHarnessContainerImage,
614-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3-fnapi:2.2.0'))
615-
else:
616-
self.assertEqual(
617-
env.proto.workerPools[0].workerHarnessContainerImage,
618-
(
619-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY +
620-
'/python%d%d-fnapi:2.2.0' %
621-
(sys.version_info[0], sys.version_info[1])))
579+
self.assertEqual(
580+
env.proto.workerPools[0].workerHarnessContainerImage,
581+
(
582+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY +
583+
'/python%d%d-fnapi:2.2.0' %
584+
(sys.version_info[0], sys.version_info[1])))
622585

623586
# batch, legacy pipeline.
624587
pipeline_options = PipelineOptions(
@@ -628,20 +591,11 @@ def test_worker_harness_image_tag_matches_released_sdk_version(self):
628591
pipeline_options,
629592
'2.0.0', #any environment version
630593
FAKE_PIPELINE_URL)
631-
if sys.version_info[0] == 2:
632-
self.assertEqual(
633-
env.proto.workerPools[0].workerHarnessContainerImage,
634-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python:2.2.0'))
635-
elif sys.version_info[0:2] == (3, 5):
636-
self.assertEqual(
637-
env.proto.workerPools[0].workerHarnessContainerImage,
638-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3:2.2.0'))
639-
else:
640-
self.assertEqual(
641-
env.proto.workerPools[0].workerHarnessContainerImage,
642-
(
643-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:2.2.0' %
644-
(sys.version_info[0], sys.version_info[1])))
594+
self.assertEqual(
595+
env.proto.workerPools[0].workerHarnessContainerImage,
596+
(
597+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:2.2.0' %
598+
(sys.version_info[0], sys.version_info[1])))
645599

646600
@mock.patch(
647601
'apache_beam.runners.dataflow.internal.apiclient.'
@@ -656,21 +610,12 @@ def test_worker_harness_image_tag_matches_base_sdk_version_of_an_rc(self):
656610
pipeline_options,
657611
'2.0.0', #any environment version
658612
FAKE_PIPELINE_URL)
659-
if sys.version_info[0] == 2:
660-
self.assertEqual(
661-
env.proto.workerPools[0].workerHarnessContainerImage,
662-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python-fnapi:2.2.0'))
663-
elif sys.version_info[0:2] == (3, 5):
664-
self.assertEqual(
665-
env.proto.workerPools[0].workerHarnessContainerImage,
666-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3-fnapi:2.2.0'))
667-
else:
668-
self.assertEqual(
669-
env.proto.workerPools[0].workerHarnessContainerImage,
670-
(
671-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY +
672-
'/python%d%d-fnapi:2.2.0' %
673-
(sys.version_info[0], sys.version_info[1])))
613+
self.assertEqual(
614+
env.proto.workerPools[0].workerHarnessContainerImage,
615+
(
616+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY +
617+
'/python%d%d-fnapi:2.2.0' %
618+
(sys.version_info[0], sys.version_info[1])))
674619

675620
# batch, legacy pipeline.
676621
pipeline_options = PipelineOptions(
@@ -680,20 +625,11 @@ def test_worker_harness_image_tag_matches_base_sdk_version_of_an_rc(self):
680625
pipeline_options,
681626
'2.0.0', #any environment version
682627
FAKE_PIPELINE_URL)
683-
if sys.version_info[0] == 2:
684-
self.assertEqual(
685-
env.proto.workerPools[0].workerHarnessContainerImage,
686-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python:2.2.0'))
687-
elif sys.version_info[0:2] == (3, 5):
688-
self.assertEqual(
689-
env.proto.workerPools[0].workerHarnessContainerImage,
690-
(names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python3:2.2.0'))
691-
else:
692-
self.assertEqual(
693-
env.proto.workerPools[0].workerHarnessContainerImage,
694-
(
695-
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:2.2.0' %
696-
(sys.version_info[0], sys.version_info[1])))
628+
self.assertEqual(
629+
env.proto.workerPools[0].workerHarnessContainerImage,
630+
(
631+
names.DATAFLOW_CONTAINER_IMAGE_REPOSITORY + '/python%d%d:2.2.0' %
632+
(sys.version_info[0], sys.version_info[1])))
697633

698634
def test_worker_harness_override_takes_precedence_over_sdk_defaults(self):
699635
# streaming, fnapi pipeline.
@@ -952,9 +888,8 @@ def test_use_unified_worker(self):
952888

953889
def test_get_response_encoding(self):
954890
encoding = apiclient.get_response_encoding()
955-
version_to_encoding = {3: 'utf8', 2: None}
956891

957-
assert encoding == version_to_encoding[sys.version_info[0]]
892+
assert encoding == 'utf8'
958893

959894
@unittest.skip("Enable once BEAM-1080 is fixed.")
960895
def test_graph_is_uploaded(self):

sdks/python/apache_beam/runners/direct/consumer_tracking_pipeline_visitor_test.py

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -45,10 +45,6 @@ class ConsumerTrackingPipelineVisitorTest(unittest.TestCase):
4545
def setUp(self):
4646
self.pipeline = Pipeline(DirectRunner())
4747
self.visitor = ConsumerTrackingPipelineVisitor()
48-
try: # Python 2
49-
self.assertCountEqual = self.assertItemsEqual
50-
except AttributeError: # Python 3
51-
pass
5248

5349
def test_root_transforms(self):
5450
root_read = beam.Impulse()

sdks/python/apache_beam/runners/interactive/background_caching_job_test.py

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020

2121
from __future__ import absolute_import
2222

23-
import sys
2423
import unittest
2524

2625
import apache_beam as beam
@@ -85,8 +84,6 @@ def _setup_test_streaming_cache(pipeline):
8584
@unittest.skipIf(
8685
not ie.current_env().is_interactive_ready,
8786
'[interactive] dependency is not installed.')
88-
@unittest.skipIf(
89-
sys.version_info < (3, 6), 'The tests require at least Python 3.6 to work.')
9087
class BackgroundCachingJobTest(unittest.TestCase):
9188
def tearDown(self):
9289
ie.new_env()

sdks/python/apache_beam/runners/interactive/cache_manager.py

Lines changed: 2 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,9 @@
2323

2424
import collections
2525
import os
26-
import sys
2726
import tempfile
28-
import urllib
27+
from urllib.parse import quote
28+
from urllib.parse import unquote_to_bytes
2929

3030
import apache_beam as beam
3131
from apache_beam import coders
@@ -34,13 +34,6 @@
3434
from apache_beam.io import tfrecordio
3535
from apache_beam.transforms import combiners
3636

37-
if sys.version_info[0] > 2:
38-
unquote_to_bytes = urllib.parse.unquote_to_bytes
39-
quote = urllib.parse.quote
40-
else:
41-
unquote_to_bytes = urllib.unquote # pylint: disable=deprecated-urllib-function
42-
quote = urllib.quote # pylint: disable=deprecated-urllib-function
43-
4437

4538
class CacheManager(object):
4639
"""Abstract class for caching PCollections.

sdks/python/apache_beam/runners/interactive/caching/streaming_cache.py

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,10 @@
2626
import time
2727
import traceback
2828
from collections import OrderedDict
29+
# We don't have an explicit pathlib dependency because this code only works with
30+
# the interactive target installed which has an indirect dependency on pathlib
31+
# through ipython>=5.9.0.
32+
from pathlib import Path
2933

3034
from google.protobuf.message import DecodeError
3135

@@ -39,14 +43,6 @@
3943
from apache_beam.testing.test_stream import ReverseTestStream
4044
from apache_beam.utils import timestamp
4145

42-
# We don't have an explicit pathlib dependency because this code only works with
43-
# the interactive target installed which has an indirect dependency on pathlib
44-
# and pathlib2 through ipython>=5.9.0.
45-
try:
46-
from pathlib import Path
47-
except ImportError:
48-
from pathlib2 import Path # python 2 backport
49-
5046
_LOGGER = logging.getLogger(__name__)
5147

5248

sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020

2121
from __future__ import absolute_import
2222

23-
import sys
2423
import unittest
2524

2625
import pytz
@@ -54,8 +53,6 @@
5453
@unittest.skipIf(
5554
not ie.current_env().is_interactive_ready,
5655
'[interactive] dependency is not installed.')
57-
@unittest.skipIf(
58-
sys.version_info < (3, 6), 'The tests require at least Python 3.6 to work.')
5956
class PCollectionVisualizationTest(unittest.TestCase):
6057
def setUp(self):
6158
ie.new_env()

0 commit comments

Comments
 (0)