lokeshj1703 opened a new pull request, #17890:
URL: https://github.com/apache/hudi/pull/17890

   ### Describe the issue this Pull Request addresses
   
   The PR cherry-picks https://github.com/apache/hudi/pull/11187 for release 
0.14.2. 
   
   ### Summary and Changelog
   
   Copied from the PR linked above
   
   We are looking to fix two problems by adding per log file marker.
   a. i. MOR data table rollbacks missed to sync original log files from failed 
commit to MDT.
   a.ii. Along these lines, if rollback instant is retried multiple times, any 
log files added from failed rollback attempts should also be synced to MDT.
   b. If there are spurious log files created even w/ successful commits, we 
need to ensure these spurious log files are also synced to MDT.
   
   So, to fix all of the above, we are adding per log file marker. Any log file 
added or appended to will create markers. We don't really need to distinguish 
between create and append and so we will go w/ APPEND IoType for markers.
   
   Fix for (a. i): Any log file added will emit a marker. If the commit of 
interest failed, hudi will trigger a rollback. During rollback planning, using 
markers we identify the original log files added by the failed commit and track 
it as part of the rollback plan. This also gets tracked in 
HoodieRollbackMetadata (had to upgrade the schema for this purpose).
   
   Fix for (a.ii): Whenever a rollback is triggered, hudi adds a rollback 
command block. With this patch, we are also emitting markers for such log 
files. During rollback execution, apart from adding log files added by failed 
commit to HoodieRollbackMetadata, we also add these log files which could have 
been added by previous attempts of rollback for the same instant.
   
   Fix for (b): During marker based reconciliation step, we check for log files 
from markers and compare it against HoodieCommitMetadata's HoodieWriteStat. If 
for any additional files tracked using markers (which could happen due to spark 
retries), we will add new HoodieWriteStat and update HoodieCommitMetadata. So, 
that when this syncs to MDT, we don't miss to track this spurious log files. We 
will use https://github.com/apache/hudi/pull/9545 to skip such spurious log 
files on the reader side. So, on the writer side, we just want to ensure we 
don't miss to track any log file created by hudi.
   
   Note: Please do note that the reconciliation for log files is kind of 
opposite of what happens w/ data files. w/ data files, any extraneous files are 
deleted. But for any extaneous log files, we can't afford to delete. Since 
there could be a concurrent reader trying to read the the file slice of 
interest. Eventually during execution, it might parse the log block header and 
might skip if its partially failed commit or inflight commit. Anyways, in 
short, we can't afford to delete any log files at any point in time except 
cleaner. So, for any extraneous log files detected, we fix the 
HoodieCommitMetadata to track these additional log files as well.
   
   Notes to reviewers to assist in reviewing:
   
   I will break down diff set of changes and the classes to review for the same.
   
       Adding per log file marker for regular log files: Added a 
callback(AppendLogWriteCallback) for this purpose since we may not know the log 
file name upfront (unlike data files). New apis are introduced to Markers for 
this purpose.
       Check files HoodieWriteHandle, HoodieAppendHandle, 
HoodieLogFormatWriter, HoodieLogFormat, HoodieLogFileWriterCallback
       DirectWriterMarkers, TimelineServerBasedMarkers, WriteMarkers, 
RequestHandler.
   
       Schema upgrade for rollback metadata.
       Check file HoodieRollbackMetadata.avsc
   
       Rollback from DT when synced to MDT: w/ the schema upgrade, we will 
fetch "logFilesFromFailedCommit" in HoodieRollbackMetadata and make a delta 
commit to MDT.
       HoodieBackedTableMetadataWriter and HoodieTableMetadataUtil.
   
   4.Rollback plan changes:
   When using Marker based rollback strategy, we poll markers to find the log 
files added. Apart from log file names, we also need the actual size. So, we do 
fs listing to fetch the file lengths. We track these as part of 
HoodieRollbackRequest.logFilesWithBlocksToRollback. There are chances that some 
files could be missing which are tracked in markers. We can ignore these files 
since this could happen (just after creating marker file, lets say the process 
crashed w/o creating the actual log files)
   Classes to check: MarkerBasedRollbackStrategy
   
       New argument to StorageScheme named listStatusUnfriendly. Depending on 
storage scheme, the file system based listing to triage the actual size of the 
log files of interest could change. As per this patch, we only have one way to 
do this. But in a follow up patch, we might have to fix that.
   
       Fixing HoodieCommitMetadata to include HoodieWriteStat for any missing 
log files which are extraneous through markers.
       Classes to check: SparkRDDWriteClient.commit and 
addMissingLogFileIfNeeded(). This also includes the file system listing to 
fetch the actual size of spurious log files if any.
   
       Rollback execution:
       a. With this patch, we are also emitting markers for rollback command 
blocks (log files).
       b. During execution, we also need to fetch any log files added by 
previous attempts of rollback and update HoodieRollbackSat if need be. Note 
that we can have only one HoodieRollbackPlan per partition.
       Classes: BaseRollbackHelper.
   
       Misc:
       HoodiePairData to add join() support. HoodieListPairData, 
HoodieJavaPairRDD.
       FsUtils.getFileStatusesUnderPartition to assist in fetching file 
statuses for some interested log files.
   
   Testing:
   
   Below tests are covered in this PR. Here, we are generating log files with 
marker and injecting failures and then validating using fs view
   
       simple insert directly to log files followed by compaction
       rollback after failure before updating MDT
       rollback after failure during updating MDT
       rollback after failure after updating MDT
       insert followed by upsert and then failed rollback
       same as above and then reattempt rollback
   
   Apart from these, we have a bunch of other tests covering metadata 
validation for the general workflows.
   
   Long running error injection tests:
   
       MOR w/ async cleaning, compaction, MDT enabled, no RLI
       MOR w/ async cleaning, compaction, inline clustering, MDT enabled, no RLI
       MOR w/ async cleaning, compaction, MDT enabled, RLI enabled
       MOR w/ async cleaning, compaction, inline clustering MDT enabled, RLI 
enabled
   
   ### Impact
   
   MDT in sync with filesystem
   
   ### Risk Level
   
   high
   
   ### Documentation Update
   
   NA
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's 
guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Enough context is provided in the sections above
   - [ ] Adequate tests were added if applicable
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to