jolshan commented on code in PR #14470:
URL: https://github.com/apache/kafka/pull/14470#discussion_r1342935092
##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1141,13 +1134,13 @@ class UnifiedLog(@volatile var logStartOffset: Long,
// Also indicate whether we have the accurate first offset or not
if (!readFirstMessage) {
if (batch.magic >= RecordBatch.MAGIC_VALUE_V2)
- firstOffset = Optional.of(new LogOffsetMetadata(batch.baseOffset))
+ firstOffset = batch.baseOffset
lastOffsetOfFirstBatch = batch.lastOffset
readFirstMessage = true
}
// check that offsets are monotonically increasing
- if (lastOffset >= batch.lastOffset)
+ if (requireOffsetsMonotonic && lastOffset >= batch.lastOffset)
Review Comment:
We technically don't need this since the check also has `if
(requireOffsetsMonotonic && !monotonic)`
##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -787,19 +770,22 @@ class ReplicaManager(val config: KafkaConfig,
)
}
- val errorResults = errorsPerPartition.map {
- case (topicPartition, error) =>
- topicPartition -> LogAppendResult(
- LogAppendInfo.UNKNOWN_LOG_APPEND_INFO,
- Some(error.exception())
+ val allResults = localProduceResults ++ errorResults
+ val produceStatus = allResults.map { case (topicPartition, result) =>
+ topicPartition -> ProducePartitionStatus(
+ result.info.lastOffset + 1, // required offset
+ new PartitionResponse(
+ result.error,
+ result.info.firstOffset,
+ result.info.lastOffset,
+ result.info.logAppendTime,
+ result.info.logStartOffset,
+ result.info.recordErrors,
+ result.exception.map(_.getMessage).orNull
Review Comment:
Are we going to start returning the error message for other exceptions now
too? I think we will add the default exception message when previously we were
not.
--
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]