-
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
Conversation
Python: tested with simple pipeline
query lineage on pipeline result:
Java, same pipeline:
query lineage on pipeline result:
|
Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment |
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.
Some early comments. Still reviewing the whole PR.
Thanks for the prompt PR.
* report at directory level. | ||
*/ | ||
private void reportSinkLineage(List<KV<FileResult<DestinationT>, ResourceId>> outputFilenames) { | ||
if (outputFilenames.size() <= 100) { |
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 paths
-
Without 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:
The consideration here is that I wish to not introduce functionality regression (i.e. report one level above) from 2.59.0
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.
} | ||
} else { | ||
for (Metadata metadata : expandedFiles) { | ||
FileSystems.reportSourceLineage(metadata.resourceId().getCurrentDirectory()); |
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.
Is it possible that one level up dir is above 100?
say for example
a/b/**/*.avro
will match all avro files under a/b/
at any level.
a/b/1/1.avro
a/b/2/2.avro
..
a/b/100/100.avro
a/b/101/101.avro
will lead to more than 100 dirs.
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.
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 comment
The 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 comment
The reason will be displayed to describe this comment to others. Learn more.
see https://github.com/apache/beam/pull/32662/files#r1790385413 in readAll we do not know in advance the number of files, so "<= check" is not applicable here sorry this is for source
Yes, one can handle the report smarter here. And in general make this reportSourceLineage
to be more accurate -- as we have the information of full list of files here, added a TODO
@@ -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 comment
The 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 comment
The 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)
- Pro of report one level above:
result still accurate (no entry dropped) when the upstream filepattern is matching, or watching a directory (which is a common use case)
- Con of report one level above:
less granularity, obviously
- Pro of report whole path:
More granularity
- Con of report whole path
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 comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. +1 to what you have current then. Thank you.
* report at directory level. | ||
*/ | ||
private void reportSinkLineage(List<KV<FileResult<DestinationT>, ResourceId>> outputFilenames) { | ||
if (outputFilenames.size() <= 100) { |
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:
The consideration here is that I wish to not introduce functionality regression (i.e. report one level above) from 2.59.0
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.
} | ||
} else { | ||
for (Metadata metadata : expandedFiles) { | ||
FileSystems.reportSourceLineage(metadata.resourceId().getCurrentDirectory()); |
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.
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).
@@ -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 comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. +1 to what you have current then. Thank you.
FileSystems.report_sink_lineage(dst) | ||
else: | ||
dst = dst_glob | ||
sep = dst_glob.find('*') |
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.
why *?
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.
it's a file pattern to match sharded file that contains a wildcard like "prefix-*-of-10000". The wildcard was introduced here:
shard_name_format = shard_name_template.replace(match.group(0), '*') |
indeed it's not obvious, added code comment
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.
Got it, the the java representation above hide this above.
Lineage.query(result.metrics(), Lineage.Type.SINK)); | ||
} else { | ||
assertEquals( | ||
Lineage.query(result.metrics(), Lineage.Type.SOURCE).isEmpty(), |
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.
In case of V1 wouldn't this just test equality of F == F and later after service side change for v2 same?
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.
Thanks for pointing this out. After review I think we can keep TextIOIT untouched, let me testing with TextIOIT-manyfiles test
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.
confirm that this does not need to change. Ran TextIOIT-manyfiles and test passed the original assert
FileSystems.report_sink_lineage(dst) | ||
else: | ||
dst = dst_glob | ||
sep = dst_glob.find('*') |
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.
Got it, the the java representation above hide this above.
I think we should make FileIO reported lineage fit in limit by itself without dependency on StringSet metric type enforcing a limit as there is no clean and short way to do so see comment: #32650 (comment) We need to handle cases
This gives us accurate visibility in lineage in most cases except when files are spread across many folders at different level in which case we only get to know bucket which is fine. On customer demand we can improve this in later releases. (cc: @robertwb to keep in loop.) |
FileSystems.reportSourceLineage(dir, LineageLevel.TOP_LEVEL); | ||
break; | ||
} | ||
FileSystems.reportSourceLineage(dir); |
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.
Doesnt this mean that when unique dir > 100 you end up reporting first 100 dire path and then when you see 101 you report the top level leading to mix of dir path reporting and bucket reporting which can be confusing.
I think a consistent approach will be report dir only if unique dirs < 100 for all known files else report bucket only.
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.
Still working on the branch actively. Now fixed. Will ask for PTAL when test passes.
993526a
to
bedc919
Compare
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.
Thank you. LGTM.
* Report File Lineage on directory * added comments, restore lineage assert in TextIOIT * Report bucket level Lineage for files larger than 100 * fix lint
* Report File Lineage on directory * added comments, restore lineage assert in TextIOIT * Report bucket level Lineage for files larger than 100 * fix lint
supercedes #32642
FileBasedSource will report every read-in file if number of file <= 100
FileBasedSource will report every unique directory (one level up) if number of file > 100 but number of unique directory <= 100
FileBasedSource will report bucket otherwise
ReadAll will report every file if number of file <= 100
ReadAll will report bucket otherwise
FileBasedSink will report every write-to file if shards <= 100
FileBasedSink (for each destination) will report single directory to write if number of file > 100
In contrast to read, we are able to report single directory because shards are in the same directory.
Please add a meaningful description for your change here
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
addresses #123
), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>
instead.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.