Kafka
  1. Kafka
  2. KAFKA-191

Investigate removing the synchronization in Log.flush

    Details

    • Type: New Feature New Feature
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.8.0
    • Fix Version/s: None
    • Component/s: core
    • Labels:
      None

      Description

      Currently we have the following synchronization in Log.scala:
      1. append, roll, and flush all share a write lock
      2. read is non-blocking

      Unfortunately this means that flush time latency is added to appends (even if the flush is done by a background thread). To fix this we should investigate a scheme to make append and flush not block each other.

        Activity

        Hide
        Jay Kreps added a comment -

        This is not needed in post 0.8 world since flush becomes less important with replication.

        Show
        Jay Kreps added a comment - This is not needed in post 0.8 world since flush becomes less important with replication.
        Hide
        Jun Rao added a comment -

        So, I guess If append and fsync can't run in parallel, the patch won't help much.

        Show
        Jun Rao added a comment - So, I guess If append and fsync can't run in parallel, the patch won't help much.
        Hide
        Jay Kreps added a comment -

        Looks like fsync does block writes, which is kind of weak. This thread is really helpful:
        http://antirez.com/post/fsync-different-thread-useless.html

        The comments do mention that fsyncdata() (aka FileChannel.force(false)) somehow doesn't interfere, which doesn't make sense to me, as for use since we always append, FileChannel.force should do the same thing regardless of whether we force metadata or not because we are changing the file size which must be flushed.

        Show
        Jay Kreps added a comment - Looks like fsync does block writes, which is kind of weak. This thread is really helpful: http://antirez.com/post/fsync-different-thread-useless.html The comments do mention that fsyncdata() (aka FileChannel.force(false)) somehow doesn't interfere, which doesn't make sense to me, as for use since we always append, FileChannel.force should do the same thing regardless of whether we force metadata or not because we are changing the file size which must be flushed.
        Hide
        Jay Kreps added a comment -

        Neha--Sounds good. I have a patch (since it is just deleting and reordering, the code change itself is trivial), I will attach. Here are my thoughts. I think we can just remove the synchronization and re-order things so that the unflushed counter and lastFlushTime both remain valid lower bounds. It is possible that the time we set could get overwritten by another thread but it is unlikely to make any practical difference. See if you agree with that logic, I am not 100% positive.

        I am not at all sure that this will actually help performance though for two reasons. First, I think it is possible that the file itself may be synchronized. Either at the java level or the OS level. So I am not sure if one can write to the file while a flush is occurring in another thread. This may take some research to understand.

        Second, if it is possible to do parallel write and flush, I think this still may not be ideal (though maybe a good short term hack). My reasoning is that this patch only fixes the blocking behavior for the time-based flush, but my question is why would I ever want to block?

        I really see two use cases:
        1. I want every message I write immediately flushed to disk in a blocking fashion before the append() is considered completed. This corresponds to flush.interval=1 in the current system.
        2. I want to periodically flush data, which could be based on the number of messages, or time, (or theoretically based on unflushed bytes, though we haven't implemented that).

        So what I am thinking is that case (1) clearly needs to be blocking to make sense. But for any periodic flush I don't see a reason to block appends. It is true that this makes the intervals inexact, but I think that is probably fine.. For example, even if I set flush.interval=5, it is unlikely that I could actually care that it is exactly 5, I just want to flush often, say ~5 messages. (Even if I did want it exact, since we always write the full messageset, I still might not get that). So I am thinking a better long-term approach might be to have a central threadpool that handles all flushing and have that always be asynchronous. So if I set flush.interval=5, then that means the background thread is triggered every 5 messages BUT no one blocks on this. In addition to this we add an immediate.commit=true/false option to force data to be flushed in a blocking way as part of the append.

        Obviously the above only works if a parallel append and flush are possible.

        Show
        Jay Kreps added a comment - Neha--Sounds good. I have a patch (since it is just deleting and reordering, the code change itself is trivial), I will attach. Here are my thoughts. I think we can just remove the synchronization and re-order things so that the unflushed counter and lastFlushTime both remain valid lower bounds. It is possible that the time we set could get overwritten by another thread but it is unlikely to make any practical difference. See if you agree with that logic, I am not 100% positive. I am not at all sure that this will actually help performance though for two reasons. First, I think it is possible that the file itself may be synchronized. Either at the java level or the OS level. So I am not sure if one can write to the file while a flush is occurring in another thread. This may take some research to understand. Second, if it is possible to do parallel write and flush, I think this still may not be ideal (though maybe a good short term hack). My reasoning is that this patch only fixes the blocking behavior for the time-based flush, but my question is why would I ever want to block? I really see two use cases: 1. I want every message I write immediately flushed to disk in a blocking fashion before the append() is considered completed. This corresponds to flush.interval=1 in the current system. 2. I want to periodically flush data, which could be based on the number of messages, or time, (or theoretically based on unflushed bytes, though we haven't implemented that). So what I am thinking is that case (1) clearly needs to be blocking to make sense. But for any periodic flush I don't see a reason to block appends. It is true that this makes the intervals inexact, but I think that is probably fine.. For example, even if I set flush.interval=5, it is unlikely that I could actually care that it is exactly 5, I just want to flush often, say ~5 messages. (Even if I did want it exact, since we always write the full messageset, I still might not get that). So I am thinking a better long-term approach might be to have a central threadpool that handles all flushing and have that always be asynchronous. So if I set flush.interval=5, then that means the background thread is triggered every 5 messages BUT no one blocks on this. In addition to this we add an immediate.commit=true/false option to force data to be flushed in a blocking way as part of the append. Obviously the above only works if a parallel append and flush are possible.
        Hide
        Neha Narkhede added a comment -

        I'm interested in investigating this and run some perf tests to see change in IO performance

        Show
        Neha Narkhede added a comment - I'm interested in investigating this and run some perf tests to see change in IO performance

          People

          • Assignee:
            Neha Narkhede
            Reporter:
            Jay Kreps
          • Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development