-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Report File Lineage on directory #32662
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -297,9 +297,10 @@ public final List<? extends FileBasedSource<T>> split( | |
System.currentTimeMillis() - startTime, | ||
expandedFiles.size(), | ||
splitResults.size()); | ||
|
||
reportSourceLineage(expandedFiles); | ||
return splitResults; | ||
} else { | ||
FileSystems.reportSourceLineage(getSingleFileMetadata().resourceId()); | ||
if (isSplittable()) { | ||
@SuppressWarnings("unchecked") | ||
List<FileBasedSource<T>> splits = | ||
|
@@ -315,6 +316,22 @@ public final List<? extends FileBasedSource<T>> split( | |
} | ||
} | ||
|
||
/** Report source Lineage. Depend on the number of files, report full file name or only dir. */ | ||
private void reportSourceLineage(List<Metadata> expandedFiles) { | ||
if (expandedFiles.size() <= 100) { | ||
for (Metadata metadata : expandedFiles) { | ||
FileSystems.reportSourceLineage(metadata.resourceId()); | ||
} | ||
} else { | ||
for (Metadata metadata : expandedFiles) { | ||
// TODO(yathu) Currently it simply report one level up if num of files exceeded 100. | ||
// Consider more dedicated strategy (e.g. resolve common ancestor) for accurancy, and work | ||
// with metrics size limit. | ||
FileSystems.reportSourceLineage(metadata.resourceId().getCurrentDirectory()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it possible that one level up dir is above 100? say for example will lead to more than 100 dirs. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, it is absolutely possible, then it relies on #32650 to guard the stringset size. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we add the <= check here too and if > roll up one level and see if we find common ancestors in dir path where the path reported will remain less than 100 and we recursively check this till we reach the root of shortest path in the list? (just a suggestion, keeping it as is also fine). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Yes, one can handle the report smarter here. And in general make this |
||
} | ||
} | ||
} | ||
|
||
/** | ||
* Determines whether a file represented by this source is can be split into bundles. | ||
* | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -88,6 +88,7 @@ public SplitIntoRangesFn(long desiredBundleSizeBytes) { | |
@ProcessElement | ||
public void process(ProcessContext c) { | ||
MatchResult.Metadata metadata = c.element().getMetadata(); | ||
FileSystems.reportSourceLineage(metadata.resourceId().getCurrentDirectory()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This can also hit 100 or more dir limit like above in file source. But if this is best effort reporting then that is fine. When combined with limit on lineage (#32650 (comment)) it will be ok. but in that case should we leave it at file level reporting itself? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is for ReadAll and the challenge here is that we do not know how many files it is going to be, and actually here it can be used in a streaming pipeline where file metadata comes in over time. So do not have hint whether report one level above or file path is preferred (like other places)
result still accurate (no entry dropped) when the upstream filepattern is matching, or watching a directory (which is a common use case)
less granularity, obviously
More granularity
More likely to hit metrics limit, and report incomplete list of files I'm open to both. Personally I prefer the former, the reason is same as the concen in the comments of #32650 , that is, given the limitation of metrics size, I consider the preference is full information > info with less granularity > has to truncate and report incomplete results There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense. +1 to what you have current then. Thank you. |
||
if (!metadata.isReadSeekEfficient()) { | ||
c.output(KV.of(c.element(), new OffsetRange(0, metadata.sizeBytes()))); | ||
return; | ||
|
@@ -140,7 +141,6 @@ public void process(ProcessContext c) throws IOException { | |
throw e; | ||
} | ||
} | ||
FileSystems.reportSourceLineage(resourceId); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -280,9 +280,31 @@ def _check_state_for_finalize_write(self, writer_results, num_shards): | |||
|
||||
src_files.append(src) | ||||
dst_files.append(dst) | ||||
FileSystems.report_sink_lineage(dst) | ||||
|
||||
self._report_sink_lineage(dst_glob, dst_files) | ||||
return src_files, dst_files, delete_files, num_skipped | ||||
|
||||
def _report_sink_lineage(self, dst_glob, dst_files): | ||||
""" | ||||
Report sink Lineage. Report every file if number of files no more than 100, | ||||
otherwise only report at directory level. | ||||
""" | ||||
if len(dst_files) <= 100: | ||||
for dst in dst_files: | ||||
FileSystems.report_sink_lineage(dst) | ||||
else: | ||||
dst = dst_glob | ||||
# dst_glob has a wildcard for shard number (see _shard_name_template) | ||||
sep = dst_glob.find('*') | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why *? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it's a file pattern to match sharded file that contains a wildcard like "prefix-*-of-10000". The wildcard was introduced here:
indeed it's not obvious, added code comment There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it, the the java representation above hide this above. |
||||
if sep > 0: | ||||
dst = dst[:sep] | ||||
try: | ||||
dst, _ = FileSystems.split(dst) | ||||
except ValueError: | ||||
return # lineage report is fail-safe | ||||
|
||||
FileSystems.report_sink_lineage(dst) | ||||
|
||||
@check_accessible(['file_path_prefix']) | ||||
def finalize_write( | ||||
self, init_result, writer_results, unused_pre_finalize_results): | ||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Any specific reason to pick 100? It it is just a good average number then that is fine too.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I admit it's a hardcoded number which is not ideal generally. The consideration here is that I wish to not introduce functionality regression (i.e. report one level above) from 2.59.0 for use cases of small number of files, while we have to do that in general to mitigate the metrics size limit.
Speaking about the number,
finalizeDestination
is called in code pathsWithout dynamic destination, finalizeDestination is called once, on single worker.
With dynamic destination, finalizeDestination is called by finalizeAllDestination, per destination.
In the first case, it is indeed safe to set a higher number, generally (however if the filepath length is very long, it will still trigger the set size truncation (#32650)
In the second case, when there are lots of destinations, and even single destination only report full number of files when shardsize <= 100, it may still trigger #32650
That being said there is no good choice to this number that can prevent #32650 being triggered in all use cases, but I think it is good enough to avoid the functionality regression for relevant use cases (small shards, when the whole path is mostly useful)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Picking a good estimate number sounds totally reasonable. Thank you. For completeness of discussion:
Since lineage is not a launched feature we do not need to worry about regression. For cases where jobs running on 2.59.0 SDK are currently emitting full path and will switch to reporting some aggregate is fine because these metrics are not consumed by anyone as of now. Dataflow service is not reporting these as lineage and cannot until the job is restarted with lineage enabled flag.
100 sounds ok to me for now.
In general I am wondering if we can make it configurable i.e. customer who use FileIOs can specify at what n we should roll up to report at higher level i.e. if number of files is >= n report dir if number of dir is >= n report bucket. We do not need to do this now but can do in future releases with more considerations.