Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/io/gcp/bigquery_tools.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@

import fastavro
import numpy as np
import regex

import apache_beam
from apache_beam import coders
Expand All @@ -70,6 +69,7 @@

# Protect against environments where bigquery library is not available.
try:
import regex
from apitools.base.py.exceptions import HttpError
from apitools.base.py.exceptions import HttpForbiddenError
from apitools.base.py.transfer import Upload
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,14 +32,17 @@
from typing import Tuple
from typing import Union

import pydot

import apache_beam as beam
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners.interactive import interactive_environment as ie
from apache_beam.runners.interactive import pipeline_instrument as inst
from apache_beam.runners.interactive.display import pipeline_graph_renderer

try:
import pydot
except ImportError:
pass

# pylint does not understand context
# pylint:disable=dangerous-default-value

Expand Down
7 changes: 5 additions & 2 deletions sdks/python/apache_beam/yaml/json_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,15 @@
from typing import Any
from typing import Optional

import jsonschema

import apache_beam as beam
from apache_beam.portability.api import schema_pb2
from apache_beam.typehints import schemas

try:
import jsonschema
except ImportError:
pass

JSON_ATOMIC_TYPES_TO_BEAM = {
'boolean': schema_pb2.BOOLEAN,
'integer': schema_pb2.INT64,
Expand Down
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/yaml/main_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,11 @@

from apache_beam.yaml import main

try:
import jsonschema
except ImportError:
jsonschema = None

TEST_PIPELINE = '''
pipeline:
type: chain
Expand Down Expand Up @@ -79,6 +84,7 @@
'''


@unittest.skipIf(jsonschema is None, "Yaml dependencies not installed")
class MainTest(unittest.TestCase):
def test_pipeline_spec_from_file(self):
with tempfile.TemporaryDirectory() as tmpdir:
Expand Down
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/yaml/yaml_io_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,11 @@
from apache_beam.typehints import schemas as schema_utils
from apache_beam.yaml.yaml_transform import YamlTransform

try:
import jsonschema
except ImportError:
jsonschema = None


class FakeReadFromPubSub:
def __init__(
Expand Down Expand Up @@ -82,6 +87,7 @@ def __call__(self, topic, *, with_attributes, id_label, timestamp_attribute):
return AssertThat(equal_to(self._messages))


@unittest.skipIf(jsonschema is None, "Yaml dependencies not installed")
class YamlPubSubTest(unittest.TestCase):
def test_simple_read(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
Expand Down
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/yaml/yaml_mapping_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,19 @@
from apache_beam.yaml import yaml_mapping
from apache_beam.yaml.yaml_transform import YamlTransform

try:
import jsonschema
except ImportError:
jsonschema = None

DATA = [
beam.Row(label='11a', conductor=11, rank=0),
beam.Row(label='37a', conductor=37, rank=1),
beam.Row(label='389a', conductor=389, rank=2),
]


@unittest.skipIf(jsonschema is None, "Yaml dependencies not installed")
class YamlMappingTest(unittest.TestCase):
def test_basic(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
Expand Down
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/yaml/yaml_transform_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,11 @@
from apache_beam.yaml import yaml_provider
from apache_beam.yaml.yaml_transform import YamlTransform

try:
import jsonschema
except ImportError:
jsonschema = None


class CreateTimestamped(beam.PTransform):
_yaml_requires_inputs = False
Expand Down Expand Up @@ -83,6 +88,7 @@ def raise_on_big(row):
}


@unittest.skipIf(jsonschema is None, "Yaml dependencies not installed")
class YamlTransformE2ETest(unittest.TestCase):
def test_composite(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
Expand Down
1 change: 1 addition & 0 deletions sdks/python/apache_beam/yaml/yaml_transform_unit_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ def new_pipeline():
pickle_library='cloudpickle'))


@unittest.skipIf(jsonschema is None, "Yaml dependencies not installed")
class MainTest(unittest.TestCase):
def assertYaml(self, expected, result):
result = SafeLineLoader.strip_metadata(result)
Expand Down
10 changes: 5 additions & 5 deletions sdks/python/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -379,15 +379,13 @@ def get_portability_package_data():
install_requires=[
'crcmod>=1.7,<2.0',
'cryptography>=39.0.0,<48.0.0',
'orjson>=3.9.7,<4',
'fastavro>=0.23.6,<2',
'fasteners>=0.3,<1.0',
# TODO(https://github.com/grpc/grpc/issues/37710): Unpin grpc
'grpcio>=1.33.1,<2,!=1.48.0,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long
'grpcio>=1.67.0; python_version >= "3.13"',
'hdfs>=2.1.0,<3.0.0',
'httplib2>=0.8,<0.23.0',
'jsonschema>=4.0.0,<5.0.0',
'jsonpickle>=3.0.0,<4.0.0',
# numpy can have breaking changes in minor versions.
# Use a strict upper bound.
Expand All @@ -407,11 +405,9 @@ def get_portability_package_data():
# 3. Exclude protobuf 4 versions that leak memory, see:
# https://github.com/apache/beam/issues/28246
'protobuf>=3.20.3,<7.0.0.dev0,!=4.0.*,!=4.21.*,!=4.22.0,!=4.23.*,!=4.24.*', # pylint: disable=line-too-long
'pydot>=1.2.0,<2',
'python-dateutil>=2.8.0,<3',
'pytz>=2018.3',
'redis>=5.0.0,<6',
'regex>=2020.6.8',
'requests>=2.32.4,<3.0.0',
'sortedcontainers>=2.4.0',
'typing-extensions>=3.7.0',
Expand Down Expand Up @@ -509,7 +505,9 @@ def get_portability_package_data():
# --extra-index-url or --index-url in requirements.txt in
# Dataflow, which allows installing python packages from private
# Python repositories in GAR.
'keyrings.google-artifactregistry-auth'
'keyrings.google-artifactregistry-auth',
'orjson>=3.9.7,<4',
'regex>=2020.6.8',
],
'interactive': [
'facets-overview>=1.1.0,<2',
Expand All @@ -520,6 +518,7 @@ def get_portability_package_data():
# Skip version 6.1.13 due to
# https://github.com/jupyter/jupyter_client/issues/637
'jupyter-client>=6.1.11,!=6.1.13,<8.2.1',
'pydot>=1.2.0,<2',
'timeloop>=1.0.2,<2',
'nbformat>=5.0.5,<6',
'nbconvert>=6.2.0,<8',
Expand Down Expand Up @@ -577,6 +576,7 @@ def get_portability_package_data():
'virtualenv-clone>=0.5,<1.0',
# https://github.com/PiotrDabkowski/Js2Py/issues/317
'js2py>=0.74,<1; python_version<"3.12"',
'jsonschema>=4.0.0,<5.0.0',
] + dataframe_dependency,
# Keep the following dependencies in line with what we test against
# in https://github.com/apache/beam/blob/master/sdks/python/tox.ini
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/tox.ini
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ pip_pre = True
# allow apps that support color to use it.
passenv=TERM,CLOUDSDK_CONFIG,DOCKER_*,TESTCONTAINERS_*,TC_*,ALLOYDB_PASSWORD
# Set [] options for pip installation of apache-beam tarball.
extras = test,dataframe
extras = test,dataframe,yaml
# Don't warn that these commands aren't installed.
allowlist_externals =
false
Expand Down
Loading