satishd commented on a change in pull request #10478:
URL: https://github.com/apache/kafka/pull/10478#discussion_r611789255



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -255,19 +261,21 @@ case object SnapshotGenerated extends 
LogStartOffsetIncrementReason {
 @threadsafe
 class Log(@volatile private var _dir: File,
           @volatile var config: LogConfig,
+          val segments: LogSegments,

Review comment:
       What I meant was we can have `Log` takes the argument as immutable 
`LogComponents` and it can initialize the vars inside `Log` with the respective 
fields from `LogComponents`. This will also set the right access of these vars 
by not giving write access by default. 
   
   ```
   class Log(@volatile private var _dir: File,
             @volatile var config: LogConfig,
             val segments: LogSegments,
             val logComponents: LogComponents,
             scheduler: Scheduler,
             brokerTopicStats: BrokerTopicStats,
             val time: Time,
             val maxProducerIdExpirationMs: Int,
             val producerIdExpirationCheckIntervalMs: Int,
             val topicPartition: TopicPartition,
             logDirFailureChannel: LogDirFailureChannel,
             @volatile var topicId: Option[Uuid],
             val keepPartitionMetadataFile: Boolean = true) extends Logging 
with KafkaMetricsGroup {
   
   
   
     @volatile private var logStartOffset: Long = logComponents.logStartOffset
     @volatile private var recoveryPoint: Long = logComponents.recoveryPoint
     @volatile private var nextOffsetMetadata: LogOffsetMetadata = 
logComponents.nextOffsetMetadata
     @volatile var leaderEpochCache: Option[LeaderEpochFileCache] = 
logComponents.leaderEpochCache
     private val producerStateManager: ProducerStateManager = 
logComponents.producerStateManager
   ```




-- 
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.

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


Reply via email to