Skip to content

Commit e8df26f

Browse files
authored
Merge pull request #33381 Migrate lineage counters to bounded tries.
2 parents 8fee3ca + 14f7caf commit e8df26f

15 files changed

+104
-116
lines changed

sdks/python/apache_beam/io/aws/s3filesystem.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
"""S3 file system implementation for accessing files on AWS S3."""
1919

2020
# pytype: skip-file
21+
import traceback
2122

2223
from apache_beam.io.aws import s3io
2324
from apache_beam.io.filesystem import BeamIOError
@@ -315,14 +316,13 @@ def delete(self, paths):
315316
if exceptions:
316317
raise BeamIOError("Delete operation failed", exceptions)
317318

318-
def report_lineage(self, path, lineage, level=None):
319+
def report_lineage(self, path, lineage):
319320
try:
320321
components = s3io.parse_s3_path(path, object_optional=True)
321322
except ValueError:
322323
# report lineage is fail-safe
324+
traceback.print_exc()
323325
return
324-
if level == FileSystem.LineageLevel.TOP_LEVEL or \
325-
(len(components) > 1 and components[-1] == ''):
326-
# bucket only
326+
if components and not components[-1]:
327327
components = components[:-1]
328-
lineage.add('s3', *components)
328+
lineage.add('s3', *components, last_segment_sep='/')

sdks/python/apache_beam/io/aws/s3filesystem_test.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -272,7 +272,8 @@ def test_lineage(self):
272272
def _verify_lineage(self, uri, expected_segments):
273273
lineage_mock = mock.MagicMock()
274274
self.fs.report_lineage(uri, lineage_mock)
275-
lineage_mock.add.assert_called_once_with("s3", *expected_segments)
275+
lineage_mock.add.assert_called_once_with(
276+
"s3", *expected_segments, last_segment_sep='/')
276277

277278

278279
if __name__ == '__main__':

sdks/python/apache_beam/io/azure/blobstoragefilesystem.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
"""Azure Blob Storage Implementation for accesing files on
1919
Azure Blob Storage.
2020
"""
21+
import traceback
2122

2223
from apache_beam.io.azure import blobstorageio
2324
from apache_beam.io.filesystem import BeamIOError
@@ -317,15 +318,14 @@ def delete(self, paths):
317318
if exceptions:
318319
raise BeamIOError("Delete operation failed", exceptions)
319320

320-
def report_lineage(self, path, lineage, level=None):
321+
def report_lineage(self, path, lineage):
321322
try:
322323
components = blobstorageio.parse_azfs_path(
323324
path, blob_optional=True, get_account=True)
324325
except ValueError:
325326
# report lineage is fail-safe
327+
traceback.print_exc()
326328
return
327-
if level == FileSystem.LineageLevel.TOP_LEVEL \
328-
or(len(components) > 1 and components[-1] == ''):
329-
# bucket only
329+
if components and not components[-1]:
330330
components = components[:-1]
331-
lineage.add('abs', *components)
331+
lineage.add('abs', *components, last_segment_sep='/')

sdks/python/apache_beam/io/azure/blobstoragefilesystem_test.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -330,7 +330,8 @@ def test_lineage(self):
330330
def _verify_lineage(self, uri, expected_segments):
331331
lineage_mock = mock.MagicMock()
332332
self.fs.report_lineage(uri, lineage_mock)
333-
lineage_mock.add.assert_called_once_with("abs", *expected_segments)
333+
lineage_mock.add.assert_called_once_with(
334+
"abs", *expected_segments, last_segment_sep='/')
334335

335336

336337
if __name__ == '__main__':

sdks/python/apache_beam/io/filebasedsink.py

Lines changed: 6 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -286,24 +286,16 @@ def _check_state_for_finalize_write(self, writer_results, num_shards):
286286

287287
def _report_sink_lineage(self, dst_glob, dst_files):
288288
"""
289-
Report sink Lineage. Report every file if number of files no more than 100,
290-
otherwise only report at directory level.
289+
Report sink Lineage. Report every file if number of files no more than 10,
290+
otherwise only report glob.
291291
"""
292-
if len(dst_files) <= 100:
292+
# There is rollup at the higher level, but this loses glob information.
293+
# Better to report multiple globs than just the parent directory.
294+
if len(dst_files) <= 10:
293295
for dst in dst_files:
294296
FileSystems.report_sink_lineage(dst)
295297
else:
296-
dst = dst_glob
297-
# dst_glob has a wildcard for shard number (see _shard_name_template)
298-
sep = dst_glob.find('*')
299-
if sep > 0:
300-
dst = dst[:sep]
301-
try:
302-
dst, _ = FileSystems.split(dst)
303-
except ValueError:
304-
return # lineage report is fail-safe
305-
306-
FileSystems.report_sink_lineage(dst)
298+
FileSystems.report_sink_lineage(dst_glob)
307299

308300
@check_accessible(['file_path_prefix'])
309301
def finalize_write(

sdks/python/apache_beam/io/filebasedsource.py

Lines changed: 2 additions & 51 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@
3939
from apache_beam.io import range_trackers
4040
from apache_beam.io.filesystem import CompressionTypes
4141
from apache_beam.io.filesystem import FileMetadata
42-
from apache_beam.io.filesystem import FileSystem
4342
from apache_beam.io.filesystems import FileSystems
4443
from apache_beam.io.restriction_trackers import OffsetRange
4544
from apache_beam.options.value_provider import StaticValueProvider
@@ -170,37 +169,11 @@ def _get_concat_source(self) -> concat_source.ConcatSource:
170169
splittable=splittable)
171170
single_file_sources.append(single_file_source)
172171

173-
self._report_source_lineage(files_metadata)
172+
FileSystems.report_source_lineage(pattern)
174173
self._concat_source = concat_source.ConcatSource(single_file_sources)
175174

176175
return self._concat_source
177176

178-
def _report_source_lineage(self, files_metadata):
179-
"""
180-
Report source Lineage. depend on the number of files, report full file
181-
name, only dir, or only top level
182-
"""
183-
if len(files_metadata) <= 100:
184-
for file_metadata in files_metadata:
185-
FileSystems.report_source_lineage(file_metadata.path)
186-
else:
187-
size_track = set()
188-
for file_metadata in files_metadata:
189-
if len(size_track) >= 100:
190-
FileSystems.report_source_lineage(
191-
file_metadata.path, level=FileSystem.LineageLevel.TOP_LEVEL)
192-
return
193-
194-
try:
195-
base, _ = FileSystems.split(file_metadata.path)
196-
except ValueError:
197-
pass
198-
else:
199-
size_track.add(base)
200-
201-
for base in size_track:
202-
FileSystems.report_source_lineage(base)
203-
204177
def open_file(self, file_name):
205178
return FileSystems.open(
206179
file_name,
@@ -382,7 +355,7 @@ def process(self, element: Union[str, FileMetadata], *args,
382355
match_results = FileSystems.match([element])
383356
metadata_list = match_results[0].metadata_list
384357
for metadata in metadata_list:
385-
self._report_source_lineage(metadata.path)
358+
FileSystems.report_source_lineage(metadata.path)
386359

387360
splittable = (
388361
self._splittable and _determine_splittability_from_compression_type(
@@ -397,28 +370,6 @@ def process(self, element: Union[str, FileMetadata], *args,
397370
metadata,
398371
OffsetRange(0, range_trackers.OffsetRangeTracker.OFFSET_INFINITY))
399372

400-
def _report_source_lineage(self, path):
401-
"""
402-
Report source Lineage. Due to the size limit of Beam metrics, report full
403-
file name or only top level depend on the number of files.
404-
405-
* Number of files<=100, report full file paths;
406-
407-
* Otherwise, report top level only.
408-
"""
409-
if self._size_track is None:
410-
self._size_track = set()
411-
elif len(self._size_track) == 0:
412-
FileSystems.report_source_lineage(
413-
path, level=FileSystem.LineageLevel.TOP_LEVEL)
414-
return
415-
416-
self._size_track.add(path)
417-
FileSystems.report_source_lineage(path)
418-
419-
if len(self._size_track) >= 100:
420-
self._size_track.clear()
421-
422373

423374
class _ReadRange(DoFn):
424375
def __init__(

sdks/python/apache_beam/io/filesystem.py

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -934,11 +934,7 @@ def delete(self, paths):
934934
"""
935935
raise NotImplementedError
936936

937-
class LineageLevel:
938-
FILE = 'FILE'
939-
TOP_LEVEL = 'TOP_LEVEL'
940-
941-
def report_lineage(self, path, unused_lineage, level=None):
937+
def report_lineage(self, path, unused_lineage):
942938
"""
943939
Report Lineage metrics for path.
944940

sdks/python/apache_beam/io/filesystems.py

Lines changed: 5 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -391,27 +391,21 @@ def get_chunk_size(path):
391391
return filesystem.CHUNK_SIZE
392392

393393
@staticmethod
394-
def report_source_lineage(path, level=None):
394+
def report_source_lineage(path):
395395
"""
396-
Report source :class:`~apache_beam.metrics.metric.LineageLevel`.
396+
Report source :class:`~apache_beam.metrics.metric.Lineage`.
397397
398398
Args:
399399
path: string path to be reported.
400-
level: the level of file path. default to
401-
:class:`~apache_beam.io.filesystem.FileSystem.LineageLevel`.FILE.
402400
"""
403-
filesystem = FileSystems.get_filesystem(path)
404-
filesystem.report_lineage(path, Lineage.sources(), level=level)
401+
FileSystems.get_filesystem(path).report_lineage(path, Lineage.sources())
405402

406403
@staticmethod
407-
def report_sink_lineage(path, level=None):
404+
def report_sink_lineage(path):
408405
"""
409406
Report sink :class:`~apache_beam.metrics.metric.Lineage`.
410407
411408
Args:
412409
path: string path to be reported.
413-
level: the level of file path. default to
414-
:class:`~apache_beam.io.filesystem.FileSystem.Lineage`.FILE.
415410
"""
416-
filesystem = FileSystems.get_filesystem(path)
417-
filesystem.report_lineage(path, Lineage.sinks(), level=level)
411+
FileSystems.get_filesystem(path).report_lineage(path, Lineage.sinks())

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1163,7 +1163,7 @@ def split(self, desired_bundle_size, start_position=None, stop_position=None):
11631163
self.table_reference.datasetId,
11641164
self.table_reference.tableId)
11651165
Lineage.sources().add(
1166-
"bigquery",
1166+
'bigquery',
11671167
self.table_reference.projectId,
11681168
self.table_reference.datasetId,
11691169
self.table_reference.tableId)

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626

2727
# pytype: skip-file
2828

29+
import traceback
2930
from typing import BinaryIO # pylint: disable=unused-import
3031

3132
from apache_beam.io.filesystem import BeamIOError
@@ -366,14 +367,13 @@ def delete(self, paths):
366367
if exceptions:
367368
raise BeamIOError("Delete operation failed", exceptions)
368369

369-
def report_lineage(self, path, lineage, level=None):
370+
def report_lineage(self, path, lineage):
370371
try:
371372
components = gcsio.parse_gcs_path(path, object_optional=True)
372373
except ValueError:
373374
# report lineage is fail-safe
375+
traceback.print_exc()
374376
return
375-
if level == FileSystem.LineageLevel.TOP_LEVEL \
376-
or(len(components) > 1 and components[-1] == ''):
377-
# bucket only
377+
if components and not components[-1]:
378378
components = components[:-1]
379-
lineage.add('gcs', *components)
379+
lineage.add('gcs', *components, last_segment_sep='/')

0 commit comments

Comments
 (0)