[ https://issues.apache.org/jira/browse/HUDI-1138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17351505#comment-17351505 ]
Ethan Guo commented on HUDI-1138: --------------------------------- Here is my plan for improving the marker file mechanism: * Abstraction of Marker file mechanism (MarkerFiles class) ** Provide an interface for marker file strategy, containing different operations (marker create, delete, etc.). ** The current scheme of directly operating on per-file markers in the file system will be one strategy. ** The marker file strategy can be configured. * New marker file strategy via timeline service ** Instead of creating per-file markers, we maintain a single MARKERS file at the commit level to store all file paths. We also maintain a MARKERS.version to indicate the version. ** The single MARKERS file is maintained at the timeline service. Any marker file operations will go through the timeline service. ** Each executor talks to the timeline service through a new endpoint, e.g., /createMarker. A new handler in the timeline service needs to be implemented to realize the R/W operations. * Handling of concurrency for consistency ** When about to finalize the write in the Hudi table, stop creating more markers in MARKERS file and let running tasks pass, i.e., do not perform writes when the marker cannot be created based on timeline service. ** When using S3, use overwrite operation for MARKERS file, and batch requests within an interval, say a few hundred milliseconds (configurable). ** Clean up logic in reconcileAgainstMarkers() which is no longer need in the new strategy. I'm going to have an end-to-end PoC working and then add all the abstraction and configurations. > Re-implement marker files via timeline server > --------------------------------------------- > > Key: HUDI-1138 > URL: https://issues.apache.org/jira/browse/HUDI-1138 > Project: Apache Hudi > Issue Type: Improvement > Components: Writer Core > Affects Versions: 0.9.0 > Reporter: Vinoth Chandar > Assignee: Ethan Guo > Priority: Blocker > Fix For: 0.9.0 > > > Even as you can argue that RFC-15/consolidated metadata, removes the need for > deleting partial files written due to spark task failures/stage retries. It > will still leave extra files inside the table (and users will pay for it > every month) and we need the marker mechanism to be able to delete these > partial files. > Here we explore if we can improve the current marker file mechanism, that > creates one marker file per data file written, by > Delegating the createMarker() call to the driver/timeline server, and have it > create marker metadata into a single file handle, that is flushed for > durability guarantees > > P.S: I was tempted to think Spark listener mechanism can help us deal with > failed tasks, but it has no guarantees. the writer job could die without > deleting a partial file. i.e it can improve things, but cant provide > guarantees -- This message was sent by Atlassian Jira (v8.3.4#803005)