From 9395a9648b501578b173a2ccf2f0a2b5525b2779 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Sat, 24 Sep 2022 23:41:46 -0400 Subject: [PATCH 1/5] Python SpannerIO Fixes * Fix spannerio read unit test actually not assert anything * Loose upperbound of spanner client library --- .../io/gcp/experimental/spannerio.py | 7 +- .../io/gcp/experimental/spannerio_test.py | 422 ++++++++++-------- sdks/python/setup.py | 3 +- 3 files changed, 242 insertions(+), 190 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py index 39a3a27aaf89..b104ebefca19 100644 --- a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py +++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py @@ -898,7 +898,12 @@ class _Mutator(namedtuple('_Mutator', @property def byte_size(self): - return self.mutation.ByteSize() + if hasattr(self.mutation, '_pb'): + # google-cloud-spanner 3.x + return self.mutation._pb.ByteSize() + else: + # google-cloud-spanner 1.x + return self.mutation.ByteSize() class MutationGroup(deque): diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py index 574ced60361c..0e22041dbea4 100644 --- a/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py +++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py @@ -82,51 +82,65 @@ def _generate_test_data(): class SpannerReadTest(unittest.TestCase): def test_read_with_query_batch( self, mock_batch_snapshot_class, mock_client_class): - mock_snapshot = mock.MagicMock() - mock_snapshot.generate_query_batches.return_value = [{ + mock_snapshot_instance = mock.MagicMock() + mock_snapshot_instance.generate_query_batches.return_value = [{ 'query': { 'sql': 'SELECT * FROM users' }, 'partition': 'test_partition' } for _ in range(3)] - mock_snapshot.process_query_batch.side_effect = [ + mock_snapshot_instance.to_dict.return_value = {} + + mock_batch_snapshot_instance = mock.MagicMock() + # Prepare process_query_batch return results for three pipelines + mock_batch_snapshot_instance.process_query_batch.side_effect = [ FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:] - ] + ] * 3 + mock_client_class.return_value.instance.return_value.database.return_value \ + .batch_snapshot.return_value = mock_snapshot_instance + mock_batch_snapshot_class.from_dict.return_value \ + = mock_batch_snapshot_instance ro = [ReadOperation.query("Select * from users")] - pipeline = TestPipeline() - - read = ( - pipeline - | 'read' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - sql="SELECT * FROM users")) - - readall = ( - pipeline - | 'read all' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - read_operations=ro)) - - readpipeline = ( - pipeline - | 'create reads' >> beam.Create(ro) - | 'reads' >> ReadFromSpanner( - TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) - - pipeline.run() - assert_that(read, equal_to(FAKE_ROWS), label='checkRead') - assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') - assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + with TestPipeline() as pipeline: + read = ( + pipeline + | 'read' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + sql="SELECT * FROM users")) + assert_that(read, equal_to(FAKE_ROWS), label='checkRead') + + with TestPipeline() as pipeline: + readall = ( + pipeline + | 'read all' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + read_operations=ro)) + assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') + + with TestPipeline() as pipeline: + readpipeline = ( + pipeline + | 'create reads' >> beam.Create(ro) + | 'reads' >> ReadFromSpanner( + TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) + assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + + # three pipelines + self.assertEqual( + mock_snapshot_instance.generate_query_batches.call_count, 3) + # three pipelines, each called three times + self.assertEqual( + mock_batch_snapshot_instance.process_query_batch.call_count, 3 * 3) def test_read_with_table_batch( self, mock_batch_snapshot_class, mock_client_class): - mock_snapshot = mock.MagicMock() - mock_snapshot.generate_read_batches.return_value = [{ + mock_snapshot_instance = mock.MagicMock() + mock_snapshot_instance.generate_read_batches.return_value = [{ 'read': { 'table': 'users', 'keyset': { @@ -137,55 +151,68 @@ def test_read_with_table_batch( }, 'partition': 'test_partition' } for _ in range(3)] - mock_snapshot.process_read_batch.side_effect = [ + mock_snapshot_instance.to_dict.return_value = {} + + mock_batch_snapshot_instance = mock.MagicMock() + # Prepare process_read_batch return results for three pipelines + mock_batch_snapshot_instance.process_read_batch.side_effect = [ FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:] - ] + ] * 3 - ro = [ReadOperation.table("users", ["Key", "Value"])] - pipeline = TestPipeline() - - read = ( - pipeline - | 'read' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - table="users", - columns=["Key", "Value"])) - - readall = ( - pipeline - | 'read all' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - read_operations=ro)) - - readpipeline = ( - pipeline - | 'create reads' >> beam.Create(ro) - | 'reads' >> ReadFromSpanner( - TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) - - pipeline.run() - assert_that(read, equal_to(FAKE_ROWS), label='checkRead') - assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') - assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + mock_client_class.return_value.instance.return_value.database.return_value \ + .batch_snapshot.return_value = mock_snapshot_instance + mock_batch_snapshot_class.from_dict.return_value \ + = mock_batch_snapshot_instance - with self.assertRaises(ValueError): - # Test the exception raised when user passes the read operations in the - # constructor and also in the pipeline. + ro = [ReadOperation.table("users", ["Key", "Value"])] + with TestPipeline() as pipeline: + read = ( + pipeline + | 'read' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + table="users", + columns=["Key", "Value"])) + assert_that(read, equal_to(FAKE_ROWS), label='checkRead') + + with TestPipeline() as pipeline: + readall = ( + pipeline + | 'read all' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + read_operations=ro)) + assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') + + with TestPipeline() as pipeline: + readpipeline = ( + pipeline + | 'create reads' >> beam.Create(ro) + | 'reads' >> ReadFromSpanner( + TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) + assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + + # three pipelines + self.assertEqual(mock_snapshot_instance.generate_read_batches.call_count, 3) + # three pipelines, each called three times + self.assertEqual( + mock_batch_snapshot_instance.process_read_batch.call_count, 3 * 3) + + with TestPipeline() as pipeline, self.assertRaises(ValueError): + # Test the exception raised at pipeline construction time, when user + # passes the read operations in the constructor and also in the pipeline _ = ( pipeline | 'reads error' >> ReadFromSpanner( project_id=TEST_PROJECT_ID, instance_id=TEST_INSTANCE_ID, database_id=_generate_database_name(), table="users")) - pipeline.run() def test_read_with_index(self, mock_batch_snapshot_class, mock_client_class): - mock_snapshot = mock.MagicMock() - mock_snapshot.generate_read_batches.return_value = [{ + mock_snapshot_instance = mock.MagicMock() + mock_snapshot_instance.generate_read_batches.return_value = [{ 'read': { 'table': 'users', 'keyset': { @@ -196,136 +223,157 @@ def test_read_with_index(self, mock_batch_snapshot_class, mock_client_class): }, 'partition': 'test_partition' } for _ in range(3)] - mock_snapshot.process_read_batch.side_effect = [ + + mock_batch_snapshot_instance = mock.MagicMock() + # Prepare process_read_batch return results for three pipelines + mock_batch_snapshot_instance.process_read_batch.side_effect = [ FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:] - ] + ] * 3 + + mock_snapshot_instance.to_dict.return_value = {} + + mock_client_class.return_value.instance.return_value.database.return_value \ + .batch_snapshot.return_value = mock_snapshot_instance + mock_batch_snapshot_class.from_dict.return_value \ + = mock_batch_snapshot_instance + ro = [ReadOperation.table("users", ["Key", "Value"], index="Key")] - pipeline = TestPipeline() - read = ( - pipeline - | 'read' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - table="users", - columns=["Key", "Value"])) - readall = ( - pipeline - | 'read all' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - read_operations=ro)) - readpipeline = ( - pipeline - | 'create reads' >> beam.Create(ro) - | 'reads' >> ReadFromSpanner( - TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) - pipeline.run() - assert_that(read, equal_to(FAKE_ROWS), label='checkRead') - assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') - assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') - with self.assertRaises(ValueError): - # Test the exception raised when user passes the read operations in the - # constructor and also in the pipeline. + with TestPipeline() as pipeline: + read = ( + pipeline + | 'read' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + table="users", + columns=["Key", "Value"])) + assert_that(read, equal_to(FAKE_ROWS), label='checkRead') + + with TestPipeline() as pipeline: + readall = ( + pipeline + | 'read all' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + read_operations=ro)) + assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll') + + with TestPipeline() as pipeline: + readpipeline = ( + pipeline + | 'create reads' >> beam.Create(ro) + | 'reads' >> ReadFromSpanner( + TEST_PROJECT_ID, TEST_INSTANCE_ID, _generate_database_name())) + assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + + # three pipelines + self.assertEqual(mock_snapshot_instance.generate_read_batches.call_count, 3) + # three pipelines, each called three times + self.assertEqual( + mock_batch_snapshot_instance.process_read_batch.call_count, 3 * 3) + + with TestPipeline() as pipeline, self.assertRaises(ValueError): + # Test the exception raised at pipeline construction time, when user + # passes the read operations in the constructor and also in the pipeline. _ = ( pipeline | 'reads error' >> ReadFromSpanner( project_id=TEST_PROJECT_ID, instance_id=TEST_INSTANCE_ID, database_id=_generate_database_name(), table="users")) - pipeline.run() def test_read_with_transaction( self, mock_batch_snapshot_class, mock_client_class): - mock_client = mock.MagicMock() - mock_instance = mock.MagicMock() - mock_database = mock.MagicMock() - mock_snapshot = mock.MagicMock() - - mock_client_class.return_value = mock_client - mock_client.instance.return_value = mock_instance - mock_instance.database.return_value = mock_database - mock_database.batch_snapshot.return_value = mock_snapshot - mock_batch_snapshot_class.return_value = mock_snapshot - mock_batch_snapshot_class.from_dict.return_value = mock_snapshot - mock_snapshot.to_dict.return_value = FAKE_TRANSACTION_INFO - - mock_session = mock.MagicMock() - mock_transaction_ctx = mock.MagicMock() - mock_transaction = mock.MagicMock() - - mock_snapshot._get_session.return_value = mock_session - mock_session.transaction.return_value = mock_transaction - mock_transaction.__enter__.return_value = mock_transaction_ctx - mock_transaction_ctx.execute_sql.return_value = FAKE_ROWS + mock_snapshot_instance = mock.MagicMock() + mock_snapshot_instance.to_dict.return_value = FAKE_TRANSACTION_INFO + + mock_transaction_instance = mock.MagicMock() + mock_transaction_instance.execute_sql.return_value = FAKE_ROWS + mock_transaction_instance.read.return_value = FAKE_ROWS + + mock_client_class.return_value.instance.return_value.database.return_value \ + .batch_snapshot.return_value = mock_snapshot_instance + mock_client_class.return_value.instance.return_value.database.return_value \ + .session.return_value.transaction.return_value.__enter__.return_value \ + = mock_transaction_instance ro = [ReadOperation.query("Select * from users")] - p = TestPipeline() - transaction = ( - p | create_transaction( - project_id=TEST_PROJECT_ID, - instance_id=TEST_INSTANCE_ID, - database_id=_generate_database_name(), - exact_staleness=datetime.timedelta(seconds=10))) + with TestPipeline() as p: + transaction = ( + p | create_transaction( + project_id=TEST_PROJECT_ID, + instance_id=TEST_INSTANCE_ID, + database_id=_generate_database_name(), + exact_staleness=datetime.timedelta(seconds=10))) - read_query = ( - p | 'with query' >> ReadFromSpanner( - project_id=TEST_PROJECT_ID, - instance_id=TEST_INSTANCE_ID, - database_id=_generate_database_name(), - transaction=transaction, - sql="Select * from users")) + read_query = ( + p | 'with query' >> ReadFromSpanner( + project_id=TEST_PROJECT_ID, + instance_id=TEST_INSTANCE_ID, + database_id=_generate_database_name(), + transaction=transaction, + sql="Select * from users")) + assert_that(read_query, equal_to(FAKE_ROWS), label='checkQuery') - read_table = ( - p | 'with table' >> ReadFromSpanner( - project_id=TEST_PROJECT_ID, - instance_id=TEST_INSTANCE_ID, - database_id=_generate_database_name(), - transaction=transaction, - table="users", - columns=["Key", "Value"])) + read_table = ( + p | 'with table' >> ReadFromSpanner( + project_id=TEST_PROJECT_ID, + instance_id=TEST_INSTANCE_ID, + database_id=_generate_database_name(), + transaction=transaction, + table="users", + columns=["Key", "Value"])) + assert_that(read_table, equal_to(FAKE_ROWS), label='checkTable') - read_indexed_table = ( - p | 'with index' >> ReadFromSpanner( - project_id=TEST_PROJECT_ID, - instance_id=TEST_INSTANCE_ID, - database_id=_generate_database_name(), - transaction=transaction, - table="users", - index="Key", - columns=["Key", "Value"])) - - read = ( - p | 'read all' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - transaction=transaction, - read_operations=ro)) - - read_pipeline = ( - p - | 'create read operations' >> beam.Create(ro) - | 'reads' >> ReadFromSpanner( - TEST_PROJECT_ID, - TEST_INSTANCE_ID, - _generate_database_name(), - transaction=transaction)) - - p.run() - - assert_that(read_query, equal_to(FAKE_ROWS), label='checkQuery') - assert_that(read_table, equal_to(FAKE_ROWS), label='checkTable') - assert_that( - read_indexed_table, equal_to(FAKE_ROWS), label='checkTableIndex') - assert_that(read, equal_to(FAKE_ROWS), label='checkReadAll') - assert_that(read_pipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + read_indexed_table = ( + p | 'with index' >> ReadFromSpanner( + project_id=TEST_PROJECT_ID, + instance_id=TEST_INSTANCE_ID, + database_id=_generate_database_name(), + transaction=transaction, + table="users", + index="Key", + columns=["Key", "Value"])) + assert_that( + read_indexed_table, equal_to(FAKE_ROWS), label='checkTableIndex') + + read = ( + p | 'read all' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + transaction=transaction, + read_operations=ro)) + assert_that(read, equal_to(FAKE_ROWS), label='checkReadAll') - with self.assertRaises(ValueError): - # Test the exception raised when user passes the read operations in the - # constructor and also in the pipeline. + read_pipeline = ( + p + | 'create read operations' >> beam.Create(ro) + | 'reads' >> ReadFromSpanner( + TEST_PROJECT_ID, + TEST_INSTANCE_ID, + _generate_database_name(), + transaction=transaction)) + assert_that(read_pipeline, equal_to(FAKE_ROWS), label='checkReadPipeline') + + # transaction setup once + self.assertEqual(mock_snapshot_instance.to_dict.call_count, 1) + # three pipelines called execute_sql + self.assertEqual(mock_transaction_instance.execute_sql.call_count, 3) + # two pipelines called read + self.assertEqual(mock_transaction_instance.read.call_count, 2) + + with TestPipeline() as p, self.assertRaises(ValueError): + # Test the exception raised at pipeline construction time, when user + # passes the read operations in the constructor and also in the pipeline. + transaction = ( + p | create_transaction( + project_id=TEST_PROJECT_ID, + instance_id=TEST_INSTANCE_ID, + database_id=_generate_database_name(), + exact_staleness=datetime.timedelta(seconds=10))) _ = ( p | 'create read operations2' >> beam.Create(ro) @@ -335,12 +383,11 @@ def test_read_with_transaction( _generate_database_name(), transaction=transaction, read_operations=ro)) - p.run() def test_invalid_transaction( self, mock_batch_snapshot_class, mock_client_class): - with self.assertRaises(ValueError): - p = TestPipeline() + # test exception raises at pipeline execution time + with self.assertRaises(ValueError), TestPipeline() as p: transaction = ( p | beam.Create([{ "invalid": "transaction" @@ -352,7 +399,6 @@ def test_invalid_transaction( database_id=_generate_database_name(), transaction=transaction, sql="Select * from users")) - p.run() def test_display_data(self, *args): dd_sql = ReadFromSpanner( diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 8017c1b692a7..13b60105e0ec 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -289,7 +289,8 @@ def get_portability_package_data(): 'google-cloud-bigquery-storage>=2.6.3,<2.14', 'google-cloud-core>=0.28.1,<3', 'google-cloud-bigtable>=0.31.1,<2', - 'google-cloud-spanner>=1.13.0,<2', + # google-cloud-spanner 2.x causes dependency parsing backoff + 'google-cloud-spanner>=1.13.0,!=2,<=3.21.0', 'grpcio-gcp>=0.2.2,<1', # GCP Packages required by ML functionality 'google-cloud-dlp>=3.0.0,<4', From eb87d9ceefb3965d65b2e6376d6f662225cce627 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Sun, 25 Sep 2022 15:17:00 -0400 Subject: [PATCH 2/5] Fix test_harness_monitoring_infos_and_metadata actually clear metrics --- sdks/python/apache_beam/metrics/execution.py | 5 +++++ sdks/python/apache_beam/runners/worker/sdk_worker_test.py | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py index 0b404de87296..4202f7996c7f 100644 --- a/sdks/python/apache_beam/metrics/execution.py +++ b/sdks/python/apache_beam/metrics/execution.py @@ -318,6 +318,11 @@ def to_runner_api_monitoring_infos(self, transform_id): def reset(self): # type: () -> None + + """Reset all metrics in the MetricsContainer. This does not delete added + metrics. + """ + for metric in self.metrics.values(): metric.reset() 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 05263aee96cf..207e377c683e 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py @@ -211,7 +211,7 @@ def get_responses(self, instruction_requests): def test_harness_monitoring_infos_and_metadata(self): # Clear the process wide metric container. - MetricsEnvironment.process_wide_container().reset() + MetricsEnvironment.process_wide_container().metrics = {} # Create a process_wide metric. urn = 'my.custom.urn' labels = {'key': 'value'} From 0a7fff4566ab9ca1129bf6019f2d9dc1b1df1acd Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 28 Sep 2022 14:48:44 -0400 Subject: [PATCH 3/5] Set google-cloud-spanner major version upper bound --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 13b60105e0ec..c80d13814e23 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -290,7 +290,7 @@ def get_portability_package_data(): 'google-cloud-core>=0.28.1,<3', 'google-cloud-bigtable>=0.31.1,<2', # google-cloud-spanner 2.x causes dependency parsing backoff - 'google-cloud-spanner>=1.13.0,!=2,<=3.21.0', + 'google-cloud-spanner>=1.13.0,!=2,<4', 'grpcio-gcp>=0.2.2,<1', # GCP Packages required by ML functionality 'google-cloud-dlp>=3.0.0,<4', From 5b42e71860c524c09c92dc4b10d1d517f9448995 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 28 Sep 2022 15:13:29 -0400 Subject: [PATCH 4/5] Test pinning google-cloud-spanner version 3 --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index c80d13814e23..096a958d513d 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -290,7 +290,7 @@ def get_portability_package_data(): 'google-cloud-core>=0.28.1,<3', 'google-cloud-bigtable>=0.31.1,<2', # google-cloud-spanner 2.x causes dependency parsing backoff - 'google-cloud-spanner>=1.13.0,!=2,<4', + 'google-cloud-spanner>=3.0.0,<4', 'grpcio-gcp>=0.2.2,<1', # GCP Packages required by ML functionality 'google-cloud-dlp>=3.0.0,<4', From 55cc71e438988086497fc103ca78ff74e918e127 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 28 Sep 2022 19:31:05 -0400 Subject: [PATCH 5/5] Revert "Test pinning google-cloud-spanner version 3" This reverts commit 5b42e71860c524c09c92dc4b10d1d517f9448995. --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 096a958d513d..c80d13814e23 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -290,7 +290,7 @@ def get_portability_package_data(): 'google-cloud-core>=0.28.1,<3', 'google-cloud-bigtable>=0.31.1,<2', # google-cloud-spanner 2.x causes dependency parsing backoff - 'google-cloud-spanner>=3.0.0,<4', + 'google-cloud-spanner>=1.13.0,!=2,<4', 'grpcio-gcp>=0.2.2,<1', # GCP Packages required by ML functionality 'google-cloud-dlp>=3.0.0,<4',