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

Skip to content

Commit 8cfd308

Browse files
authored
Revert #27878 (#27968)
* Merge in master * Remove changes.md callout
1 parent 761aa7f commit 8cfd308

27 files changed

Lines changed: 5758 additions & 537 deletions

CHANGES.md

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,6 @@
5959

6060
## I/Os
6161

62-
* Python GCSIO is now implemented with GCP GCS Client instead of apitools ([#25676](https://github.com/apache/beam/issues/25676))
6362
* Java KafkaIO now supports picking up topics via topicPattern ([#26948](https://github.com/apache/beam/pull/26948))
6463
* Support for read from Cosmos DB Core SQL API ([#23604](https://github.com/apache/beam/issues/23604))
6564
* Upgraded to HBase 2.5.5 for HBaseIO. (Java) ([#27711](https://github.com/apache/beam/issues/19554))

playground/frontend/playground_components/assets/symbols/python.g.yaml

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4790,6 +4790,10 @@ GBKTransform:
47904790
- from_runner_api_parameter
47914791
- to_runner_api_parameter
47924792
GcpTestIOError: {}
4793+
GcsDownloader:
4794+
methods:
4795+
- get_range
4796+
- size
47934797
GCSFileSystem:
47944798
methods:
47954799
- checksum
@@ -4833,6 +4837,10 @@ GcsIOError: {}
48334837
GcsIOOverrides:
48344838
methods:
48354839
- retry_func
4840+
GcsUploader:
4841+
methods:
4842+
- finish
4843+
- put
48364844
GeneralPurposeConsumerSet:
48374845
methods:
48384846
- flush

sdks/python/apache_beam/examples/complete/game/user_score.py

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,6 @@ def format_user_score_sums(user_score):
177177
(user, score) = user_score
178178
return 'user: %s, total_score: %s' % (user, score)
179179

180-
181180
( # pylint: disable=expression-not-assigned
182181
p
183182
| 'ReadInputText' >> beam.io.ReadFromText(args.input)

sdks/python/apache_beam/internal/gcp/auth.py

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -111,9 +111,6 @@ def __getattr__(self, attr):
111111
"""Delegate attribute access to underlying google-auth credentials."""
112112
return getattr(self._google_auth_credentials, attr)
113113

114-
def get_google_auth_credentials(self):
115-
return self._google_auth_credentials
116-
117114

118115
class _Credentials(object):
119116
_credentials_lock = threading.Lock()
@@ -122,7 +119,7 @@ class _Credentials(object):
122119

123120
@classmethod
124121
def get_service_credentials(cls, pipeline_options):
125-
# type: (PipelineOptions) -> Optional[_ApitoolsCredentialsAdapter]
122+
# type: (PipelineOptions) -> Optional[google.auth.credentials.Credentials]
126123
with cls._credentials_lock:
127124
if cls._credentials_init:
128125
return cls._credentials
@@ -142,7 +139,7 @@ def get_service_credentials(cls, pipeline_options):
142139

143140
@staticmethod
144141
def _get_service_credentials(pipeline_options):
145-
# type: (PipelineOptions) -> Optional[_ApitoolsCredentialsAdapter]
142+
# type: (PipelineOptions) -> Optional[google.auth.credentials.Credentials]
146143
if not _GOOGLE_AUTH_AVAILABLE:
147144
_LOGGER.warning(
148145
'Unable to find default credentials because the google-auth library '

sdks/python/apache_beam/io/gcp/bigquery_test.py

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -827,7 +827,6 @@ def test_streaming_inserts_flush_on_byte_size_limit(self, mock_insert):
827827
exception_type=exceptions.ServiceUnavailable if exceptions else None,
828828
error_message='backendError')
829829
])
830-
@unittest.skip('Not compatible with new GCS client. See GH issue #26334.')
831830
def test_load_job_exception(self, exception_type, error_message):
832831

833832
with mock.patch.object(bigquery_v2_client.BigqueryV2.JobsService,
@@ -867,7 +866,6 @@ def test_load_job_exception(self, exception_type, error_message):
867866
exception_type=exceptions.InternalServerError if exceptions else None,
868867
error_message='internalError'),
869868
])
870-
@unittest.skip('Not compatible with new GCS client. See GH issue #26334.')
871869
def test_copy_load_job_exception(self, exception_type, error_message):
872870

873871
from apache_beam.io.gcp import bigquery_file_loads
@@ -886,7 +884,7 @@ def test_copy_load_job_exception(self, exception_type, error_message):
886884
mock.patch.object(BigQueryWrapper,
887885
'wait_for_bq_job'), \
888886
mock.patch('apache_beam.io.gcp.internal.clients'
889-
'.storage.storage_v1_client.StorageV1.ObjectsService'),\
887+
'.storage.storage_v1_client.StorageV1.ObjectsService'), \
890888
mock.patch('time.sleep'), \
891889
self.assertRaises(Exception) as exc, \
892890
beam.Pipeline() as p:

sdks/python/apache_beam/io/gcp/bigquery_tools.py

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -339,9 +339,6 @@ class BigQueryWrapper(object):
339339
offer a common place where retry logic for failures can be controlled.
340340
In addition, it offers various functions used both in sources and sinks
341341
(e.g., find and create tables, query a table, etc.).
342-
343-
Note that client parameter in constructor is only for testing purposes and
344-
should not be used in production code.
345342
"""
346343

347344
# If updating following names, also update the corresponding pydocs in
@@ -356,7 +353,6 @@ def __init__(self, client=None, temp_dataset_id=None, temp_table_ref=None):
356353
self.gcp_bq_client = client or gcp_bigquery.Client(
357354
client_info=ClientInfo(
358355
user_agent="apache-beam-%s" % apache_beam.__version__))
359-
360356
self._unique_row_id = 0
361357
# For testing scenarios where we pass in a client we do not want a
362358
# randomized prefix for row IDs.

sdks/python/apache_beam/io/gcp/bigquery_tools_test.py

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@
6060
from apitools.base.py.exceptions import HttpError, HttpForbiddenError
6161
from google.api_core.exceptions import ClientError, DeadlineExceeded
6262
from google.api_core.exceptions import InternalServerError
63+
import google.cloud
6364
except ImportError:
6465
ClientError = None
6566
DeadlineExceeded = None
@@ -223,6 +224,23 @@ def test_delete_dataset_retries_for_timeouts(self, patched_time_sleep):
223224
wrapper._delete_dataset('', '')
224225
self.assertTrue(client.datasets.Delete.called)
225226

227+
@unittest.skipIf(
228+
google and not hasattr(google.cloud, '_http'), # pylint: disable=c-extension-no-member
229+
'Dependencies not installed')
230+
@mock.patch('time.sleep', return_value=None)
231+
@mock.patch('google.cloud._http.JSONConnection.http')
232+
def test_user_agent_insert_all(self, http_mock, patched_sleep):
233+
wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper()
234+
try:
235+
wrapper._insert_all_rows('p', 'd', 't', [{'name': 'any'}], None)
236+
except: # pylint: disable=bare-except
237+
# Ignore errors. The errors come from the fact that we did not mock
238+
# the response from the API, so the overall insert_all_rows call fails
239+
# soon after the BQ API is called.
240+
pass
241+
call = http_mock.request.mock_calls[-2]
242+
self.assertIn('apache-beam-', call[2]['headers']['User-Agent'])
243+
226244
@mock.patch('time.sleep', return_value=None)
227245
def test_delete_table_retries_for_timeouts(self, patched_time_sleep):
228246
client = mock.Mock()

sdks/python/apache_beam/io/gcp/gcsfilesystem.py

Lines changed: 25 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -254,13 +254,24 @@ def rename(self, source_file_names, destination_file_names):
254254
gcs_batches.append(gcs_current_batch)
255255

256256
# Execute GCS renames if any and return exceptions.
257-
try:
258-
for batch in gcs_batches:
259-
self._gcsIO().copy_batch(batch)
260-
self._gcsIO().delete_batch(source_file_names)
257+
exceptions = {}
258+
for batch in gcs_batches:
259+
copy_statuses = self._gcsIO().copy_batch(batch)
260+
copy_succeeded = []
261+
for src, dest, exception in copy_statuses:
262+
if exception:
263+
exceptions[(src, dest)] = exception
264+
else:
265+
copy_succeeded.append((src, dest))
266+
delete_batch = [src for src, dest in copy_succeeded]
267+
delete_statuses = self._gcsIO().delete_batch(delete_batch)
268+
for i, (src, exception) in enumerate(delete_statuses):
269+
dest = copy_succeeded[i][1]
270+
if exception:
271+
exceptions[(src, dest)] = exception
261272

262-
except Exception as exception:
263-
raise BeamIOError("Rename operation failed", exception)
273+
if exceptions:
274+
raise BeamIOError("Rename operation failed", exceptions)
264275

265276
def exists(self, path):
266277
"""Check if the provided path exists on the FileSystem.
@@ -329,7 +340,8 @@ def metadata(self, path):
329340
"""
330341
try:
331342
file_metadata = self._gcsIO()._status(path)
332-
return FileMetadata(path, file_metadata['size'], file_metadata['updated'])
343+
return FileMetadata(
344+
path, file_metadata['size'], file_metadata['last_updated'])
333345
except Exception as e: # pylint: disable=broad-except
334346
raise BeamIOError("Metadata operation failed", {path: e})
335347

@@ -348,7 +360,12 @@ def _delete_path(path):
348360
else:
349361
path_to_use = path
350362
match_result = self.match([path_to_use])[0]
351-
self._gcsIO().delete_batch([m.path for m in match_result.metadata_list])
363+
statuses = self._gcsIO().delete_batch(
364+
[m.path for m in match_result.metadata_list])
365+
# pylint: disable=used-before-assignment
366+
failures = [e for (_, e) in statuses if e is not None]
367+
if failures:
368+
raise failures[0]
352369

353370
exceptions = {}
354371
for path in paths:

sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -272,16 +272,25 @@ def test_rename_error(self, mock_gcsio):
272272
'gs://bucket/to2',
273273
'gs://bucket/to3',
274274
]
275-
gcsio_mock.delete_batch.side_effect = Exception("BadThings")
275+
exception = IOError('Failed')
276+
gcsio_mock.delete_batch.side_effect = [[(f, exception) for f in sources]]
276277
gcsio_mock.copy_batch.side_effect = [[
277278
('gs://bucket/from1', 'gs://bucket/to1', None),
278279
('gs://bucket/from2', 'gs://bucket/to2', None),
279280
('gs://bucket/from3', 'gs://bucket/to3', None),
280281
]]
281282

282283
# Issue batch rename.
283-
with self.assertRaisesRegex(BeamIOError, r'^Rename operation failed'):
284+
expected_results = {
285+
(s, d): exception
286+
for s, d in zip(sources, destinations)
287+
}
288+
289+
# Issue batch rename.
290+
with self.assertRaisesRegex(BeamIOError,
291+
r'^Rename operation failed') as error:
284292
self.fs.rename(sources, destinations)
293+
self.assertEqual(error.exception.exception_details, expected_results)
285294

286295
gcsio_mock.copy_batch.assert_called_once_with([
287296
('gs://bucket/from1', 'gs://bucket/to1'),
@@ -299,7 +308,7 @@ def test_delete(self, mock_gcsio):
299308
# Prepare mocks.
300309
gcsio_mock = mock.MagicMock()
301310
gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock
302-
gcsio_mock._status.return_value = {'size': 0, 'updated': 99999.0}
311+
gcsio_mock._status.return_value = {'size': 0, 'last_updated': 99999.0}
303312
files = [
304313
'gs://bucket/from1',
305314
'gs://bucket/from2',
@@ -317,7 +326,7 @@ def test_delete_error(self, mock_gcsio):
317326
gcsfilesystem.gcsio.GcsIO = lambda pipeline_options=None: gcsio_mock
318327
exception = IOError('Failed')
319328
gcsio_mock.delete_batch.side_effect = exception
320-
gcsio_mock._status.return_value = {'size': 0, 'updated': 99999.0}
329+
gcsio_mock._status.return_value = {'size': 0, 'last_updated': 99999.0}
321330
files = [
322331
'gs://bucket/from1',
323332
'gs://bucket/from2',

0 commit comments

Comments
 (0)