You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jay Kreps (JIRA)" <ji...@apache.org> on 2012/09/18 19:34:07 UTC

[jira] [Created] (KAFKA-521) Cleanup public API for Log.scala

Jay Kreps created KAFKA-521:
-------------------------------

             Summary: Cleanup public API for Log.scala
                 Key: KAFKA-521
                 URL: https://issues.apache.org/jira/browse/KAFKA-521
             Project: Kafka
          Issue Type: Improvement
            Reporter: Jay Kreps
            Priority: Minor


A few of the APIs in Log.scala and LogManager.scala have drifted a little bit. Since this is a core interface it would be nice to think these through and clean it up a bit.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13506083#comment-13506083 ] 

Neha Narkhede commented on KAFKA-521:
-------------------------------------

Thanks for v5, few things (mostly minor, except for 3.1) that I realized while reading v5 -

1. Log
1.1 The param name maxLogFileSize is invalid
1.2 The numbering in the API docs for analyzeAndValidateMessageSet regressed to all 1s :)
1.3 API doc bug above maybeFlush - @param The instead of @param numberOfMessages. Same for truncateTo() and truncateFullyAndStartAt() as well
2. LogManager
2.1 Same as 1.2 for createAndValidateLogDirs
3. LogSegment
3.1 The read API returns null if startPosition is null. Earlier it used to return empty set. At least one usage of this API depends on it returning a non-null value. For example nextOffset -

    val ms = read(index.lastOffset, None, log.sizeInBytes)
    ms.lastOption match {
      case None => baseOffset
      case Some(last) => last.nextOffset
    }

You took care of the other in Log.read. I personally prefer non null since it forces you to handle the null case, but I understand there is a minor performance implication and you probably prefer the null approach :)
4. Probably best to leave out the change to producer.num.retries. We are working on it on 0.8 and it will probably collide on the merge anyway. But if you want to leave it in, that's fine as well.


+1 otherwise
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch, KAFKA-521-v5.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13506155#comment-13506155 ] 

Neha Narkhede commented on KAFKA-521:
-------------------------------------

1.2 Tried that, looks like none of those approaches work. The only thing that worked for me was traditional wiki style syntax -
   * <ol>
   * <li>Ensure that there are no duplicates in the directory list</li>
   * <li>Create each directory if it doesn't exist</li>
   * <li>Check that each path is a readable directory</li>
   * </ol>


                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch, KAFKA-521-v5.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Swapnil Ghike (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13504392#comment-13504392 ] 

Swapnil Ghike commented on KAFKA-521:
-------------------------------------

Overall, a lot of things have become much more readable and easier to understand. I have a few comments:

1. Log.analyzeAndValidateMessageSet(): We can probably replace these three statements  
 val messageCodec = m.compressionCodec
  if(messageCodec != NoCompressionCodec)
      codec = messageCodec

with  codec = m.compressionCodec

2. There are a few unused imports in Log, LogManager.

3. Why not touch() the last segment in Log.deleteOldSegments() if  the last segment is empty? Not deleting the empty last segment is equivalent to deleting the segment and creating a new segment. If the last segment expires based on retention time and remains empty, and its last modified time is not reset in deleteOldSegments(), then it will eventually be able to hold only one message set.

4. In reference to the comment "do not lock around actual file deletion, it isn't O(1) on many filesystems" in Log.deleteOldSegments(): 

Log.truncateTo () calls deletable.foreach(_.delete()) and truncateFullyAndStartAt() calls segmentsToDelete.foreach(_.delete()) inside a synchronized block. It should be possible to move the file deletion out of the synchronized block in the second case at least, which will help if truncateFullyAndStartAt() is called by ReplicaFetcherThread.handleOffsetOutOfRange(). 

I was also not quite sure about what you meant by "Fix a bug in Log.truncateTo--we need to delete the old segments before creating the new segment to ensure we don't delete the new segment."

5. Log.roll() : 
val prev = segments.put(segment.baseOffset, segment)
 if(prev != null)
   throw new KafkaException(%s)

replaces the old segment and then checks if there was already a segment at that offset. Not sure if it matters whether the new or the old segment stays in the map in case a KafkaException is thrown.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13504815#comment-13504815 ] 

Jay Kreps commented on KAFKA-521:
---------------------------------

1. The point of the logic is to recompress all the messages in the append together using one of the given compression codecs, if there are any. This logic is a bit weird, and Neha suggested having the server set its own compression codec on a per-topic basis irrespective of the client codec, which actually does seem better. But in any the current purpose of that code is to detect the compression codec used in the message set for use during recompression. The corner case is if there are multiple compression codecs in use in the same set (totally legal). So the logic I want is that if there are 10 gzipped messages followed by 1 uncompressed message, I want to gzip all the messages together so I can't just reset the codec on each message, compression (of any sort) has to override non-compression.
2. Can you be more specific, my IDE doesn't detect these. :-(
3. Not sure if I understand what you are saying, but I was a little nervous about this change. I will follow up with you to understand better.
4. Yes. The corner case is that moving the deletion outside the synchronized block introduces the possibility of deleting the newly created segment by accident if it has the same offset as one of the segments being deleted. That is, say that truncate saves out a list of segments to delete, including one with filename X and removes these from the Map, then it creates a new segment, then it does the actual deletions. What if one of the new segments is named X? Rather than handle that case I just do it in the lock on the assumption that truncate is going to be rare and truncating a whole segment will be very rare. This is what that comment meant. One the other hand deleting old segments I expect to be frequent so I tried to properly handle that case without locking around the delete. I am not 100% confident in the solution, though.
5. Since this is basically a programmer error that should never occur I think it should be okay either way.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Neha Narkhede (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13505183#comment-13505183 ] 

Neha Narkhede commented on KAFKA-521:
-------------------------------------

Pretty good cleanup and great usage of Java data structures for the segment list stuff

1. Some new unit tests fail -
[error] Test Failed: testRecoveryFixesCorruptIndex(kafka.log.LogSegmentTest)
java.lang.NullPointerException
        at kafka.log.LogSegmentTest$$anonfun$testRecoveryFixesCorruptIndex$2.apply$mcVI$sp(LogSegmentTest.scala:175)
        at scala.collection.immutable.Range$ByOne$class.foreach$mVc$sp(Range.scala:282)
        at scala.collection.immutable.Range$$anon$2.foreach$mVc$sp(Range.scala:265)
        at kafka.log.LogSegmentTest.testRecoveryFixesCorruptIndex(LogSegmentTest.scala:174)

2. FileMessageSet
The param end says it is the absolute position in the file at which the message set ends. However, when isSlice is false, it is passed in as Int.MaxValue. Can we change the comment to reflect that ? Also, the API docs should also explain isSlice

3. KafkaApis
3.1 This is not introduced by your code, but will be good to optimize -

            if (allOffsets.exists(_ > hw))
              hw +: allOffsets.dropWhile(_ > hw)
            else
              allOffsets
This code does 2 O(n) operation, when it suffices to do just one. Just dropWhile should suffice here, it will return all offsets when none is greater than hw and will filter out the right ones that are greater than hw.
3.2 Both LogManager and ReplicaManager take in the Time object that KafkaServer was created with. This helps unit testing since we can pass in MockTime. Since we moved the getOffsetsBefore to KafkaApis, does it make sense to have it use the same time object as well ?

4. Log
4.1 To be consistent, probably makes sense to change the comments on the public APIs to use java docs conventions instead. Right now, it documents each param but doesn't add the @param tag.
4.2 Typo above logSegments() - oldes
4.3 In deleteOldSegments, it is probably better to acquire the lock before iterating over segments. If not, the list of segments that should be deleted can be stale and can lead to us inadvertently deleting segments that we newly rolled over.
4.4. We talked about this offline, but the actual delete being outside the synchronized block is another bug.
4.5 Probably a good idea to defensively check for empty segments in truncateTo since segments.firstEntry will throw a NPE
4.6 Maybe activeSegment should protect against NPE as well ? Al though not entirely sure when this can happen


5. LogSegment
5.1 The lastUpdateTime has changed to create time. So we use a create time vs the first append time to decide if a new segment should be rolled. So if no new data has been added to a segment, we still roll it ?
5.2 touch() API is unused.

6. ProducerConfig
Was the change to drop the number of retries intended ?

                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Attachment: KAFKA-521-v4.patch

New patch, v4, contains two changes:
1. Remove unused imports Swapnil pointed out
2. Further refinement of Log.deleteOldSegments.

Swapnil, check out the new deleteOldSegments impl and see if you like that better.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Attachment: KAFKA-521-v3.patch

Updated patch. Only change is to make the maxOffset parameter in Log.read consistent. Now it is always the end of the range EXCLUSIVE.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Swapnil Ghike (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13505148#comment-13505148 ] 

Swapnil Ghike commented on KAFKA-521:
-------------------------------------

Log.deleteOldSegments:

Just a nitpick, I personally find the following more easy to understand:
logSegments.takeWhile(s => predicate(s) && !(s.baseOffset == lastSegment.baseOffset && s.size == 0))

Also, deletable is not computed inside a synchronized block anymore. Do we have a guarantee that between the computation of deletable and entering the synchronized block, no segment will be deleted? Otherwise let's say numToDelete is equal to segments.size before entering the synchronized block, if a segment is deleted before deleteOldSegements enters the synchronized block then roll() won't happen.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Swapnil Ghike (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13504871#comment-13504871 ] 

Swapnil Ghike commented on KAFKA-521:
-------------------------------------

1. I see, thanks for the clarification. If there are multiple compression codecs in the same set, would it make sense to have a precedence order among them to decide which compression codec is used for compressing all the messages together? Right now it seems that the codec of the last compressed message will win.

2. If you are using IntelliJ, you can right click on the file name in the project structure and click on "Optimize Imports". The unused imports that I see are 
Log:
import kafka.api.OffsetRequest
import java.util.{Comparator, Collections, ArrayList}
import scala.math._
import kafka.server.BrokerTopicStat

LogManager:
import kafka.log.Log._

3. Sure, we can talk.

4. Yes, that was a good catch. It's also less prone to introducing new bugs this way.

I am not super confident about my understanding of the non-Log* part of this patch, so it will be good if someone else could also review that part.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Attachment: KAFKA-521-v5.patch

New patch. Addresses your comments (see below) and also one more refactoring:
1. Swap argument order in Log.read. Now it is read(start, end, size), which makes a lot more sense. The odd order was originally due to default args which I ended up not using anyway.

Your comments:
1. Ack yes this test is totally messed up. Fixed.
2. I clarified a bit. It is technically an upper bound on the end position.
3.1, 3.2. Agreed, but since we had a follow-up item to generalize that API I tried not to touch it. There is already a lot in this patch. Basically there are so many improvements we could make there that there is no point making just one or two. The performance issue shouldn't be too pressing in any case.
4.1. Done, did this throughout. Would be nice to do this globally, actually. We don't need to be dogmatic, but it is worth thinking about it.
4.2 Fixed.
4.3, 4.4. Agreed. Filed KAFKA-636 to cover all delete related issues. I will probably do that next, but I don't think we want to mix it in with this one.
4.5. Are you refering to segments.firstEntry.getValue.baseOffset? It is an invariant of the whole class that segments be non-empty. I can check it but the resulting exception will not be more informative, I think, and there is no graceful handling of this. The important thing is to validate that this invariant actually holds...
4.6 Ditto.
5.1. No, I check that. The check is (segment.size > 0 && time.milliseconds - segment.created > rollIntervalMs). This makes more sense. We had that first append time that we weirdly reset at times and were using that to infer that the segment was non-empty, it was convoluted. This is more straight forward: (1) a create time based on when the segment is instantiated (still not quite right in the case of restart), and (2) a roll criteria that says, in effect, "roll non-empty segments created more than this threshold". 
5.2 Deleted.
6. Yes. The rationale is that retries lead to duplicates which breaks our normal guarantee. I think we should default to very high timeouts and no retries and let the user tune appropriately after thinking about what they actually want. Arguably this should be done on a separate ticket.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch, KAFKA-521-v5.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Attachment: KAFKA-521-v1.patch

Changes
FileMessageSet:
- Improve docs
- rename constructor argument initChannelPositionToEnd to isSlice (i.e. describe what it means not what it does internally)
- Use isSlice to avoid calling fileChannel.size (which stats the file) for slices. This was happening on every read, which is a minor performance bug.
- Fix minor bug in FileChannel.iterator. The iterator was not working correctly in the case of a sliced message set, it respected the start argument but not the limit so it would always yield a complete log suffix. This is not too important because the iterator is only used for testing at this time.

Log:
- Remove SegmentList; replace with ConcurrentSkipListMap, a synchronized, sorted map of offset->segment. The goal of this is just reducing custom tricky code. This will mean no more possibility of duplicate entries in the segment list, which was a common problem. Likely this is not quite as efficient, but should be good enough.
- Remove Log.findRange (since we removed SegmentList)
- Move Log static/object methods to bottom of file for clarity
- Support rebuilding the index file if it is missing
- Move recoverSegment into LogSegment as LogSegment.recover since it is segment specific.
- Move the per-topic message count metric out of Log and into KafkaApis since Log is not aware of topics
- Change Log.append to return the full set of info about the append (first offset, last offset, number of messages, and compression codec) as a case class LogAppendInfo (previously MessageSetInfo). We were already calculating this as part of the append, this just exposes it. This is needed to move the per-topic monitoring out so we can still get the message count without computing it a second time.
- Cleanup the control flow in append--now it is very imperative but, I think, a lot more readable
- Log.read now supports reads that fall on the border between two segments when we have sparse offsets (i.e. if we have garbage collected messages at the end of a log segment and hence the read beings in the one segment but finds nothing and needs to then go to the next). This situation can't yet arrise since there is no cleaning, but that will happen soon.
- Combine markDeletedWhile() and deleteSegments() into a single method deleteOldSegments that both removes from the skip list and deletes the files since we always did both together.
- Remove Log.rollToOffset since the only offset we ever rolled to was logEndOffset, now we just have Log.roll()
- Add helper methods: activeSegment() and logSegments() to get either the last segment or all segments sorted by offset.
- Move getOffsetsBefore out of Log and into KafkaApis since this logic is very specific to the API and not at all general Log functionality. Instead KafkaApis uses Log.logSegments() to just directly process the segments. This is much cleaner and more generic (e.g. could support a more generalized version of that api without change).
- Refactor truncateTo to work with the new segment map
- Rename truncateAndStartWithNewOffset(offset) to truncateFullyAndStartAt(offset) and refactor to work with new segment map
- Remove topicName since Log is not knowledgable about topics

LogManager:
- Remove brokerId from LogManager since LogManager doesn't know about brokers
- Remove LogManager.getOffsets and move it into KafkaApis--see description above

SegmentList, Range, SegmentListTest: deleted

LogSegment:
- Move recover() method out of Log and into LogSegment
- No longer implement Range since we don't have to work with SegmentList
- Rename start to baseOffset for clarity
- Change firstAppendTime to created. No longer try to initialize this on the first append, instead just initialize it when the segment is created or truncated. Neither of these is quite correct (since we don't have a persistent created time), but this is not less correct and is less hacky.
- Remove deleted flag since we weren't using it
- Mark the individual methods in this class as either threadsafe or nonthreadsafe
- Change LogSegment.read() now returns null if the start position is beyond the last message in this segment. This is necessary to allow us to detect this case and skip onto the next segment in the Log. This handles the gap case discussed above under Log.
- Move deleteSegment() method out of Log and into LogSegment

MessageSet
- Change toString method. Previously it would create a string for the whole message set. This is reasonable for ByteBufferMessageSet but not for FileMessageSet. Instead limit this to the first 100 messages to avoid potential OOM due to some logging.

ProducerConfig
- Change the producer.num.retries to default to 1--we should not default to something that can cause duplicates

KafkaApis
- Move getOffsets methods out of Log into KafkaApis

FileMessageSet:
- Document all test cases

LogManagerTest
- Document all test cases

LogOffsetTest
- Move out of kafka.log and into kafka.server
- Delete the section that covers the getOffsets api since that is covered already in a getOffset-specific test

LogSegmentTest
- Document all test cases

LogTest
- Add test case to cover non-sequential offsets better
- Add a test case to cover the case where a read falls off the end of the segment (discussed above)
- Add a test case to cover the case where the index has been deleted
- Misc improvements
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13504066#comment-13504066 ] 

Jay Kreps commented on KAFKA-521:
---------------------------------

This patch is ready for review. It is intended for checkin against trunk not 0.8.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13506098#comment-13506098 ] 

Jay Kreps commented on KAFKA-521:
---------------------------------

1.1 Good catch
1.2 According to my reading of the scaladoc docs that is how you are supposed to do ordered lists. Will it take proper numbers?
1.3 Fixed.
3.1 Nice catch, fixed.
4. Changed it back. I actually want this change in 0.8. Will come by and discuss.

Checking in on trunk with those changes.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch, KAFKA-521-v3.patch, KAFKA-521-v4.patch, KAFKA-521-v5.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Description: 
There are a number of items it would be nice to cleanup in the log subsystem:
1. Misc. funky apis in Log and LogManager
2. Much of the functionality in Log should move into LogSegment along with corresponding tests
3. We should remove SegmentList and instead use a ConcurrentSkipListMap

The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.

This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

  was:A few of the APIs in Log.scala and LogManager.scala have drifted a little bit. Since this is a core interface it would be nice to think these through and clean it up a bit.

       Priority: Major  (was: Minor)
        Summary: Refactor Log subsystem  (was: Cleanup public API for Log.scala)
    
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (KAFKA-521) Refactor Log subsystem

Posted by "Jay Kreps (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/KAFKA-521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jay Kreps updated KAFKA-521:
----------------------------

    Attachment: KAFKA-521-v2.patch

Updated patch. In addition to the items in v1 This has the following changes:
1. Rebased again
2. FileMessageSet: Renamed the "limit" variable in FileMessageSet used for slicing to "end" since it was very confusing whether this was the absolute position of the final byte in the slice or the relative offset from the start position given (limit usually means the later).
3. FileMessageSegment, LogSegment, Log: Found a bug in LogSegment.recover(). If the message size was corrupted it is possible for the recovery procedure to go out of memory since it tries to load a message of the corrupt size. To fix this I now pass the max message size that we specify in the config into the recovery procedure, and in turn into FileMessageSet.iterator, and treat any message in the log larger than this maximum as a corruption.
4. Log: Fix a bug in Log.truncateTo--we need to delete the old segments before creating the new segment to ensure we don't delete the new segment.
5. LogSement: Added a new optimization to LogSegment.translateOffset. We potentially do two translations per read()--one for the startOffset and one for the end offset (if there is one). It is possible that the nearest index entry lower bound on the end offset is actually lower than the startOffset--potentially much lower. So in this case rather than starting the search from this position it is better to start from the translated startOffset since it is guaranteed to be <= endOffset. A nice special case of this is that if you fetch a single message at a time you never do more than one message read in Log.searchFor.
6. I did an assessment of unit test coverage and added test cases where I thought there were particularly glaring holes. Added cases covering: index rebuilding, log corruption, iterating a FileMessageSet slice, truncating a FileMessageSet. I also expanded a few other existing test.
                
> Refactor Log subsystem
> ----------------------
>
>                 Key: KAFKA-521
>                 URL: https://issues.apache.org/jira/browse/KAFKA-521
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jay Kreps
>         Attachments: KAFKA-521-v1.patch, KAFKA-521-v2.patch
>
>
> There are a number of items it would be nice to cleanup in the log subsystem:
> 1. Misc. funky apis in Log and LogManager
> 2. Much of the functionality in Log should move into LogSegment along with corresponding tests
> 3. We should remove SegmentList and instead use a ConcurrentSkipListMap
> The general idea of the refactoring fall into two categories. First, improve and thoroughly document the public APIs. Second, have a clear delineation of responsibility between the various layers:
> 1. LogManager is responsible for the creation and deletion of logs as well as the retention of data in log segments. LogManager is the only layer aware of partitions and topics. LogManager consists of a bunch of individual Log instances and interacts with them only through their public API (mostly true today).
> 2. Log represents a totally ordered log. Log is responsible for reading, appending, and truncating the log. A log consists of a bunch of LogSegments. Currently much of the functionality in Log should move into LogSegment with Log interacting only through the Log interface. Currently we reach around this a lot to call into FileMessageSet and OffsetIndex.
> 3. A LogSegment consists of an OffsetIndex and a FileMessageSet. It supports largely the same APIs as Log, but now localized to a single segment.
> This cleanup will simplify testing and debugging because it will make the responsibilities and guarantees at each layer more clear.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira