splett2 opened a new pull request, #14470:
URL: https://github.com/apache/kafka/pull/14470

   The `LogAppendInfo` class is a bit bloated in terms of class fields. That's 
because it is used as an umbrella class for both leader log appends and 
follower log appends and needs to carry fields for both. This makes the 
constructor for the class a bit cludgy to use. I noticed there were a few 
fields that didn't seem necessary.
   
   Below is a description of changes:
   - `firstOffset` is a `LogOffsetMetadata` but there are no readers of the 
field that use anything but the `messageOffset` field - simplified to a long.
   - `LogAppendInfo.errorMessage` is only set in one context - when calling 
`LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo`. When we use this 
constructor, we pass up the original exception in `LogAppendResult` anyway, so 
the field is redundant with the `LogAppendResult.exception` field. This allows 
us to simplify the handling in #14378 since there are no custom error messages 
we just return whatever is in the exception message.
   - We only use targetCompressionType when constructing the LogValidator - 
just inline the call instead of including it in the LogAppendInfo.
   - offsetsMonotonic is only used when not assigning offsets to throw an 
exception - just throw the exception instead of setting a field to throw later.
   - shallowCount is only there to determine whether there are any messages in 
the append. Instead, we can just check `validBytes` which is incremented with a 
non-zero value every time we increment shallowCount.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to