3939from apache_beam .io import range_trackers
4040from apache_beam .io .filesystem import CompressionTypes
4141from apache_beam .io .filesystem import FileMetadata
42- from apache_beam .io .filesystem import FileSystem
4342from apache_beam .io .filesystems import FileSystems
4443from apache_beam .io .restriction_trackers import OffsetRange
4544from 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
423374class _ReadRange (DoFn ):
424375 def __init__ (
0 commit comments