Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-545

Add a Performance Suite for the Log subsystem

    Details

    • Type: New Feature
    • Status: Closed
    • Priority: Blocker
    • Resolution: Fixed
    • Affects Version/s: 0.8.0
    • Fix Version/s: 0.8.0
    • Component/s: None
    • Labels:

      Description

      We have had several performance concerns or potential improvements for the logging subsystem. To conduct these in a data-driven way, it would be good to have a single-machine performance test that isolated the performance of the log.

      The performance optimizations we would like to evaluate include

      • Special casing appends in a follower which already have the correct offset to avoid decompression and recompression
      • Memory mapping either all or some of the segment files to improve the performance of small appends and lookups
      • Supporting multiple data directories and avoiding RAID

      Having a standalone tool is nice to isolate the component and makes profiling more intelligible.

      This test would drive load against Log/LogManager controlled by a set of command line options. These command line program could then be scripted up into a suite of tests that covered variations in message size, message set size, compression, number of partitions, etc.

      Here is a proposed usage for the tool:

      ./bin/kafka-log-perf-test.sh
      Option Description
      ------ -----------
      --partitions The number of partitions to write to
      --dir The directory in which to write the log
      --message-size The size of the messages
      --set-size The number of messages per write
      --compression Compression alg
      --messages The number of messages to write
      --readers The number of reader threads reading the data

      The tool would capture latency and throughput for the append() and read() operations.

      1. KAFKA-545-v3.patch
        39 kB
        Jay Kreps
      2. KAFKA-545-v2.patch
        39 kB
        Jay Kreps
      3. KAFKA-545-draft.patch
        35 kB
        Jay Kreps
      4. KAFKA-545.patch
        37 kB
        Jay Kreps

        Activity

        Hide
        jkreps Jay Kreps added a comment -

        Attaching wip that has this command. Usage is:

        jkreps-mn:kafka-git jkreps$ ./bin/kafka-run-class.sh kafka.perf.LogPerformance --help
        Option Description
        ------ -----------
        --batch-size <Integer: size> Number of messages to write in a
        single batch. (default: 200)
        --compression-codec <Integer: If set, messages are sent compressed
        compression codec > (default: 0)
        --date-format <date format> The date format to use for formatting
        the time field. See java.text.
        SimpleDateFormat for options.
        (default: yyyy-MM-dd HH:mm:ss:SSS)
        --dir <path> The log directory. (default:
        /var/folders/wV/wVHRnnYrEX0ZFMG7ypsUXE+++TM/-
        Tmp-/kafka-8193339)
        --flush-interval <Integer: The number of messages in a partition
        num_messages> between flushes. (default:
        2147483647)
        --flush-time <Integer: ms> The time between flushes. (default:
        2147483647)
        --help Print usage.
        --hide-header If set, skips printing the header for
        the stats
        --index-interval <Integer: bytes> The number of bytes in between index
        entries. (default: 4096)
        --message-size <Integer: size> The size of each message. (default:
        100)
        --messages <Long: count> The number of messages to send or
        consume (default:
        9223372036854775807)
        --partitions <Integer: num_partitions> The number of partitions. (default: 1)
        --reader-batch-size <Integer: The number of messages to write at
        num_messages> once. (default: 200)
        --readers <Integer: num_threads> The number of reader threads.
        (default: 1)
        --reporting-interval <Integer: size> Interval at which to print progress
        info. (default: 5000)
        --show-detailed-stats If set, stats are reported for each
        reporting interval as configured by
        reporting-interval
        --topic <topic> REQUIRED: The topic to consume from.
        --writer-batch-size <Integer: The number of messages to write at
        num_messages> once. (default: 200)
        --writers <Integer: num_threads> The number of writer threads.
        (default: 1)

        Show
        jkreps Jay Kreps added a comment - Attaching wip that has this command. Usage is: jkreps-mn:kafka-git jkreps$ ./bin/kafka-run-class.sh kafka.perf.LogPerformance --help Option Description ------ ----------- --batch-size <Integer: size> Number of messages to write in a single batch. (default: 200) --compression-codec <Integer: If set, messages are sent compressed compression codec > (default: 0) --date-format <date format> The date format to use for formatting the time field. See java.text. SimpleDateFormat for options. (default: yyyy-MM-dd HH:mm:ss:SSS) --dir <path> The log directory. (default: /var/folders/wV/wVHRnnYrEX0ZFMG7ypsUXE+++TM/- Tmp-/kafka-8193339) --flush-interval <Integer: The number of messages in a partition num_messages> between flushes. (default: 2147483647) --flush-time <Integer: ms> The time between flushes. (default: 2147483647) --help Print usage. --hide-header If set, skips printing the header for the stats --index-interval <Integer: bytes> The number of bytes in between index entries. (default: 4096) --message-size <Integer: size> The size of each message. (default: 100) --messages <Long: count> The number of messages to send or consume (default: 9223372036854775807) --partitions <Integer: num_partitions> The number of partitions. (default: 1) --reader-batch-size <Integer: The number of messages to write at num_messages> once. (default: 200) --readers <Integer: num_threads> The number of reader threads. (default: 1) --reporting-interval <Integer: size> Interval at which to print progress info. (default: 5000) --show-detailed-stats If set, stats are reported for each reporting interval as configured by reporting-interval --topic <topic> REQUIRED: The topic to consume from. --writer-batch-size <Integer: The number of messages to write at num_messages> once. (default: 200) --writers <Integer: num_threads> The number of writer threads. (default: 1)
        Hide
        jkreps Jay Kreps added a comment -

        The initial set of results show the following:

        Big impacts on the write path are
        1. flush
        2. FileChannel overhead
        3. CRC calculation

        Big impacts on the read path are
        1. FileMessageSet.searchFor
        2. MessageSet iteration (mostly an artifact of the test)

        Notable is that the index lookup doesn't show up at all.

        Since our read path is an order of magnitude faster than our write path, it makes sense to focus on writes first. The most important thing here is to handle flush better.

        The important thing to know is that on Linux fsync holds a global lock on writes to the file, so effectively flush blocks all appends (even though we only really want to flush the data already written). We are effectively targeting linux here since it is so common.

        Originally we thought that we could just disable flush and depend on replication for durability with the flush to disk happening via Linux's background pdflush process. However recovery demands a "known stable point" from which to recover. Currently we guarantee that all but the last log segment have been flushed. The problem with this is that disabling the time or size based flush effectively just prolongs the time of the global lock until the segment is full, but under high load this could mean suddenly blocking on writing out 2GB of data--a huge multi-second latency spike.

        There are a number of possible solutions to this dilemma:
        1. Memory map the FileMessageSet. It is possible that the locking characteristics for mmap/msync are different than fsync. This would have the advantage of also getting rid of the overhead of the write call which in java is pretty high and thus making FileMessageSet.searchFor and iterator much faster. There are two disadvantages of this. First we would have to pre-allocate file segments. This would likely confuse people a bit. We are already doing it with index files, but those are smaller. Second memory map eats up process address space. This would likely mean that running on a 32 bit OS would be infeasible (since you would only have 2GB).
        2. Change the recovery to recover from an arbitrary point in the log, and write out a "high water mark" in a similar way to how we do for the ISR. This would let us just avoid syncing the active segment.
        3. Move the flush into a background thread. This wouldn't help if one was using a time-based or size-based flush but would help for the flush at the time of segment roll since at that point there is guaranteed to be no more writes on the segment. This would require recovering the last two segments. This is kind of hacky but easy to implement.

        So there are two questions this raises:
        1. Does mmap/msync actually have different locking than fsync? Mmap might be the preferable solution since it solves lots of performance bottlenecks all at once.
        2. Does fsync block reads? This is a question Jun raised. We don't actually care to much about adding a little bit of latency to consumers, but with synchronous replication fetch latency is a lower bound on produce latency. So blocking reads may be as bad as blocking writes. It is unlikely that an in-sync replica would be a full segment behind, but nonetheless.

        I started to investigate these two questions.

        I started with (2). It doesn't seem that fsync blocks reads. This is intuitive. To test this I wrote a test that has one thread continually append to a file, one thread call fsync every 2 seconds, and one thread do random reads. I measured the max read time over 100k reads. Here is a trace of the output:
        flushing
        flush completed in 0.35281
        0.226049
        1.347807
        0.02298
        1.562114
        0.041638
        2.119723
        0.027258
        5.329019
        flushing
        8.32156
        flush completed in 81.123215
        0.04145
        1.473818
        0.06444
        1.733412
        0.050216
        1.437777
        0.04984
        1.612728
        0.026001
        1.858957
        0.041096
        0.390903
        flushing
        12.527883
        flush completed in 67.416953
        0.055656
        1.450987
        0.029861
        1.469376
        0.047733
        1.313674
        0.024264
        1.71214
        0.027112
        0.023717
        1.228905
        0.029688
        1.215998
        flushing
        flush completed in 59.289193
        18.217726
        1.549095
        0.029295
        1.367316
        0.047124
        1.389574
        0.034049
        0.030982
        1.129182
        0.048443
        1.070381
        0.040149
        1.07179
        flushing
        flush completed in 59.340792
        18.668898
        0.933095
        0.041071
        1.197376
        0.035512
        1.34228
        0.042432
        0.024397
        0.835786
        0.026552
        1.496774
        0.036751
        1.148597
        flushing
        flush completed in 59.308117
        16.34416
        0.854841
        0.053005
        1.013405
        0.08081
        0.051634
        1.218344
        0.015086
        1.447114
        0.019883
        1.128675
        0.041854
        1.148591
        flushing
        15.110585
        flush completed in 47.303732
        1.018977
        0.015041
        0.036324
        1.293796
        0.051184
        1.291538
        0.013544
        1.211112
        0.014241
        1.520512
        0.027815
        1.246593
        flushing
        0.016121
        flush completed in 59.38031
        22.635984
        0.051233
        0.054701
        0.712837
        0.01345
        1.004364
        0.017261
        1.216081
        0.019825
        ^C[jkreps@jkreps-ld kafka-jbod]$ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestFileChannelReadLocking 100000
        flushing
        flush completed in 0.528026
        0.217155
        1.620644
        0.034775
        0.032913
        1.867401
        0.026142
        2.367951
        0.05301
        flushing
        3.941371
        flush completed in 252.11533
        17.678379
        1.226629
        0.043844
        2.254627
        0.05084
        1.654637
        0.028077
        0.029319
        1.217127
        0.029779
        1.251271
        0.281076
        0.919776
        flushing
        0.050014
        1.156282
        flush completed in 238.852776
        5.953432
        0.038438
        0.966883
        0.048641
        0.907416
        0.037052
        1.595778
        0.023821
        0.923264
        0.047909
        0.921312
        0.058346
        0.058062
        flushing
        0.906503
        0.090477
        flush completed in 239.282906
        1.504453
        0.014805
        1.276596
        0.051536
        1.200947
        0.052367
        0.068161
        1.585333
        0.051904
        1.052337
        0.063165
        1.502294
        0.01266
        flushing
        0.839178
        0.048978
        flush completed in 290.738005
        1.414586
        0.069402
        0.056503
        1.0008
        0.050265
        0.955949
        0.050486
        1.014454
        0.048935
        1.210959
        0.054599
        1.313663
        0.058152
        flushing
        0.062286
        1.036941
        flush completed in 242.879275
        11.401024
        1.390022
        0.07374
        0.92633
        0.013332
        1.015606
        0.04448
        1.687692
        0.014552
        0.018272
        1.339258
        0.051723
        ^C

        As you can see there is some locking happening. But it is not for the duration of the flush. I tried varying the amount of data being written and the max read time remains constant. My guess is that what is happening is that the locking is at the page level, which is what we see with pdflush. This should be acceptable as the latency is bounded to the time to flush one page regardless of the flush size.

        I am working on testing mmap.

        Show
        jkreps Jay Kreps added a comment - The initial set of results show the following: Big impacts on the write path are 1. flush 2. FileChannel overhead 3. CRC calculation Big impacts on the read path are 1. FileMessageSet.searchFor 2. MessageSet iteration (mostly an artifact of the test) Notable is that the index lookup doesn't show up at all. Since our read path is an order of magnitude faster than our write path, it makes sense to focus on writes first. The most important thing here is to handle flush better. The important thing to know is that on Linux fsync holds a global lock on writes to the file, so effectively flush blocks all appends (even though we only really want to flush the data already written). We are effectively targeting linux here since it is so common. Originally we thought that we could just disable flush and depend on replication for durability with the flush to disk happening via Linux's background pdflush process. However recovery demands a "known stable point" from which to recover. Currently we guarantee that all but the last log segment have been flushed. The problem with this is that disabling the time or size based flush effectively just prolongs the time of the global lock until the segment is full, but under high load this could mean suddenly blocking on writing out 2GB of data--a huge multi-second latency spike. There are a number of possible solutions to this dilemma: 1. Memory map the FileMessageSet. It is possible that the locking characteristics for mmap/msync are different than fsync. This would have the advantage of also getting rid of the overhead of the write call which in java is pretty high and thus making FileMessageSet.searchFor and iterator much faster. There are two disadvantages of this. First we would have to pre-allocate file segments. This would likely confuse people a bit. We are already doing it with index files, but those are smaller. Second memory map eats up process address space. This would likely mean that running on a 32 bit OS would be infeasible (since you would only have 2GB). 2. Change the recovery to recover from an arbitrary point in the log, and write out a "high water mark" in a similar way to how we do for the ISR. This would let us just avoid syncing the active segment. 3. Move the flush into a background thread. This wouldn't help if one was using a time-based or size-based flush but would help for the flush at the time of segment roll since at that point there is guaranteed to be no more writes on the segment. This would require recovering the last two segments. This is kind of hacky but easy to implement. So there are two questions this raises: 1. Does mmap/msync actually have different locking than fsync? Mmap might be the preferable solution since it solves lots of performance bottlenecks all at once. 2. Does fsync block reads? This is a question Jun raised. We don't actually care to much about adding a little bit of latency to consumers, but with synchronous replication fetch latency is a lower bound on produce latency. So blocking reads may be as bad as blocking writes. It is unlikely that an in-sync replica would be a full segment behind, but nonetheless. I started to investigate these two questions. I started with (2). It doesn't seem that fsync blocks reads. This is intuitive. To test this I wrote a test that has one thread continually append to a file, one thread call fsync every 2 seconds, and one thread do random reads. I measured the max read time over 100k reads. Here is a trace of the output: flushing flush completed in 0.35281 0.226049 1.347807 0.02298 1.562114 0.041638 2.119723 0.027258 5.329019 flushing 8.32156 flush completed in 81.123215 0.04145 1.473818 0.06444 1.733412 0.050216 1.437777 0.04984 1.612728 0.026001 1.858957 0.041096 0.390903 flushing 12.527883 flush completed in 67.416953 0.055656 1.450987 0.029861 1.469376 0.047733 1.313674 0.024264 1.71214 0.027112 0.023717 1.228905 0.029688 1.215998 flushing flush completed in 59.289193 18.217726 1.549095 0.029295 1.367316 0.047124 1.389574 0.034049 0.030982 1.129182 0.048443 1.070381 0.040149 1.07179 flushing flush completed in 59.340792 18.668898 0.933095 0.041071 1.197376 0.035512 1.34228 0.042432 0.024397 0.835786 0.026552 1.496774 0.036751 1.148597 flushing flush completed in 59.308117 16.34416 0.854841 0.053005 1.013405 0.08081 0.051634 1.218344 0.015086 1.447114 0.019883 1.128675 0.041854 1.148591 flushing 15.110585 flush completed in 47.303732 1.018977 0.015041 0.036324 1.293796 0.051184 1.291538 0.013544 1.211112 0.014241 1.520512 0.027815 1.246593 flushing 0.016121 flush completed in 59.38031 22.635984 0.051233 0.054701 0.712837 0.01345 1.004364 0.017261 1.216081 0.019825 ^C [jkreps@jkreps-ld kafka-jbod] $ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestFileChannelReadLocking 100000 flushing flush completed in 0.528026 0.217155 1.620644 0.034775 0.032913 1.867401 0.026142 2.367951 0.05301 flushing 3.941371 flush completed in 252.11533 17.678379 1.226629 0.043844 2.254627 0.05084 1.654637 0.028077 0.029319 1.217127 0.029779 1.251271 0.281076 0.919776 flushing 0.050014 1.156282 flush completed in 238.852776 5.953432 0.038438 0.966883 0.048641 0.907416 0.037052 1.595778 0.023821 0.923264 0.047909 0.921312 0.058346 0.058062 flushing 0.906503 0.090477 flush completed in 239.282906 1.504453 0.014805 1.276596 0.051536 1.200947 0.052367 0.068161 1.585333 0.051904 1.052337 0.063165 1.502294 0.01266 flushing 0.839178 0.048978 flush completed in 290.738005 1.414586 0.069402 0.056503 1.0008 0.050265 0.955949 0.050486 1.014454 0.048935 1.210959 0.054599 1.313663 0.058152 flushing 0.062286 1.036941 flush completed in 242.879275 11.401024 1.390022 0.07374 0.92633 0.013332 1.015606 0.04448 1.687692 0.014552 0.018272 1.339258 0.051723 ^C As you can see there is some locking happening. But it is not for the duration of the flush. I tried varying the amount of data being written and the max read time remains constant. My guess is that what is happening is that the locking is at the page level, which is what we see with pdflush. This should be acceptable as the latency is bounded to the time to flush one page regardless of the flush size. I am working on testing mmap.
        Hide
        jkreps Jay Kreps added a comment -

        Same test now with 20 seconds worth of data accumulating:
        [jkreps@jkreps-ld kafka-jbod]$ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestFileChannelReadLocking 500000
        flushing
        flush completed in 0.497006
        2.271428
        11.766812
        1.660411
        1.861596
        2.039938
        1.278876
        1.407181
        1.130133
        1.192209
        1.663374
        1.658432
        1.124757
        1.254995
        1.848904
        1.861381
        1.158326
        1.414888
        1.240507
        1.542315
        1.543492
        1.395788
        1.128224
        1.244737
        1.323254
        1.004004
        1.508619
        1.294839
        1.237147
        1.369261
        1.500938
        1.098796
        1.140933
        1.195621
        0.825858
        1.21719
        flushing
        1.187579
        1.234125
        0.981985
        0.999659
        1.05744
        1.171083
        flush completed in 2488.938675
        1.219635
        1.240126
        1.192422
        1.604653
        1.412199
        1.89463
        1.282256
        1.08756
        1.360199
        0.947128
        1.130891
        0.782065
        1.453711
        1.225088
        1.704001
        1.110982
        1.155404
        1.297822
        1.450305
        1.224275
        1.272652
        1.280408
        1.23271
        1.144039
        1.273127
        1.302072
        1.408974
        1.348525
        1.556987
        1.193373
        1.407276
        1.722947
        1.443469
        1.751133
        flushing
        1.288651
        flush completed in 608.099163
        1.520736
        1.233443
        1.553179
        1.627624
        1.613462
        1.534873
        1.508163
        1.538743
        1.489821
        1.318509
        1.537813
        1.385722
        1.06104
        1.31107
        1.232484
        1.621071
        1.63272
        1.800139
        1.311899
        1.315283
        1.552909
        1.518307
        1.384089
        1.520744
        1.762693
        1.467796
        1.699609
        1.159155
        1.469895
        1.187978
        1.830385
        flushing
        1.669841
        1.341722
        1.52613
        flush completed in 2040.207681
        1.202133
        1.400995
        1.077904
        1.69022
        1.055655
        1.145438
        1.535375
        1.281362
        1.168067
        0.989543
        1.162816
        1.531742
        1.296389
        1.065467

        Again, more or less constant time even though now we have 2 second flushes.

        Show
        jkreps Jay Kreps added a comment - Same test now with 20 seconds worth of data accumulating: [jkreps@jkreps-ld kafka-jbod] $ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestFileChannelReadLocking 500000 flushing flush completed in 0.497006 2.271428 11.766812 1.660411 1.861596 2.039938 1.278876 1.407181 1.130133 1.192209 1.663374 1.658432 1.124757 1.254995 1.848904 1.861381 1.158326 1.414888 1.240507 1.542315 1.543492 1.395788 1.128224 1.244737 1.323254 1.004004 1.508619 1.294839 1.237147 1.369261 1.500938 1.098796 1.140933 1.195621 0.825858 1.21719 flushing 1.187579 1.234125 0.981985 0.999659 1.05744 1.171083 flush completed in 2488.938675 1.219635 1.240126 1.192422 1.604653 1.412199 1.89463 1.282256 1.08756 1.360199 0.947128 1.130891 0.782065 1.453711 1.225088 1.704001 1.110982 1.155404 1.297822 1.450305 1.224275 1.272652 1.280408 1.23271 1.144039 1.273127 1.302072 1.408974 1.348525 1.556987 1.193373 1.407276 1.722947 1.443469 1.751133 flushing 1.288651 flush completed in 608.099163 1.520736 1.233443 1.553179 1.627624 1.613462 1.534873 1.508163 1.538743 1.489821 1.318509 1.537813 1.385722 1.06104 1.31107 1.232484 1.621071 1.63272 1.800139 1.311899 1.315283 1.552909 1.518307 1.384089 1.520744 1.762693 1.467796 1.699609 1.159155 1.469895 1.187978 1.830385 flushing 1.669841 1.341722 1.52613 flush completed in 2040.207681 1.202133 1.400995 1.077904 1.69022 1.055655 1.145438 1.535375 1.281362 1.168067 0.989543 1.162816 1.531742 1.296389 1.065467 Again, more or less constant time even though now we have 2 second flushes.
        Hide
        nehanarkhede Neha Narkhede added a comment -

        Makes sense, are the reads in your test using NIO ?

        Show
        nehanarkhede Neha Narkhede added a comment - Makes sense, are the reads in your test using NIO ?
        Hide
        jkreps Jay Kreps added a comment -

        Yes, for the above data everything is done on a FileChannel.

        Show
        jkreps Jay Kreps added a comment - Yes, for the above data everything is done on a FileChannel.
        Hide
        jkreps Jay Kreps added a comment -

        Okay, wrote some tests for lockin MappedByteBuffer and FileChannel to see how writes block reads. I think these things work the same in both cases. Again the results are a series of max times, this time over 5M writes. For both cases I see behavior similar to the below--high max times while a flush is occuring but no hard locking. I am not sure the exact cause of this, but it is safe to say that mmap is no panacea here.

        [jkreps@jkreps-ld kafka-jbod]$ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestMmapLocking $((2*1024*1024*1024-1)) 5000 5000000 1
        flushing
        flush completed in 49.601333 ms
        10.077477
        2.214208
        1.712157
        1.895483
        1.798951
        1.934366
        1.738388
        flushing
        13.097689
        262.515081
        flush completed in 1752.944685 ms
        2.044426
        1.655329
        2.063751
        1.55256
        2.429741
        1.717703
        1.477672
        9.815928
        flushing
        368.168449
        flush completed in 1928.963959 ms
        240.966127
        1.600222
        1.191583
        1.750381
        2.09028
        1.694696
        1.88224
        2.122531
        flushing
        168.749728
        385.088614
        flush completed in 2160.281323 ms
        241.846812
        1.745029
        1.82718
        1.756801
        1.822239
        1.689906
        1.708812
        1.633651
        flushing
        183.764496
        368.315579
        flush completed in 2180.277532 ms
        276.418226
        1.737839
        1.730913
        1.711507
        1.540686
        2.011486
        1.937501
        1.834844
        flushing
        95.983117
        129.890815
        flush completed in 2695.239899 ms
        1288.338521
        1.828685
        1.613301
        1.63822
        1.725626

        Show
        jkreps Jay Kreps added a comment - Okay, wrote some tests for lockin MappedByteBuffer and FileChannel to see how writes block reads. I think these things work the same in both cases. Again the results are a series of max times, this time over 5M writes. For both cases I see behavior similar to the below--high max times while a flush is occuring but no hard locking. I am not sure the exact cause of this, but it is safe to say that mmap is no panacea here. [jkreps@jkreps-ld kafka-jbod] $ java -server -Xmx128M -Xms128M -XX:+UseConcMarkSweepGC -cp project/boot/scala-2.8.0/lib/scala-library.jar:core/target/scala_2.8.0/test-classes kafka.TestMmapLocking $((2*1024*1024*1024-1)) 5000 5000000 1 flushing flush completed in 49.601333 ms 10.077477 2.214208 1.712157 1.895483 1.798951 1.934366 1.738388 flushing 13.097689 262.515081 flush completed in 1752.944685 ms 2.044426 1.655329 2.063751 1.55256 2.429741 1.717703 1.477672 9.815928 flushing 368.168449 flush completed in 1928.963959 ms 240.966127 1.600222 1.191583 1.750381 2.09028 1.694696 1.88224 2.122531 flushing 168.749728 385.088614 flush completed in 2160.281323 ms 241.846812 1.745029 1.82718 1.756801 1.822239 1.689906 1.708812 1.633651 flushing 183.764496 368.315579 flush completed in 2180.277532 ms 276.418226 1.737839 1.730913 1.711507 1.540686 2.011486 1.937501 1.834844 flushing 95.983117 129.890815 flush completed in 2695.239899 ms 1288.338521 1.828685 1.613301 1.63822 1.725626
        Hide
        jkreps Jay Kreps added a comment -
        • Improve the existing TestLinearWriteSpeed.scala that does linear writes to a file. Now this prints latency/throughput info at a configurable period and allows writing to many files at once as well as using either mmap or write. This is a great test because it gives a bound on fs performance without any Kafka code involved.
        • Added a test program that drives the LogManager level. Can write to any number of partitions and controls various configs.
        • Add three test programs to test read and write locking in the OS. These are a bit hacky and I am fine leaving them on my machine too though they may come in handy again. (TestFileChannelLocking.scala, TestFileChannelReadLocking.scala, TestMmapLocking.scala)
          Misc things
        • Remove the reference to flushInterval in many tests. Lots of tests were setting this to 1 to commit messages immediately. That is no longer necessary in 0.8.
        • Added a new configuration replica.highwatermark.checkpoint.ms to control the checkpointing of hwm. Previously we used the config for log sync, but that doesn't make sense.
        • Added Log.toString method for convenience
        • Added explicit check for maxSize < 0 and startOffset > endOffset in LogSegment.read
        Show
        jkreps Jay Kreps added a comment - Improve the existing TestLinearWriteSpeed.scala that does linear writes to a file. Now this prints latency/throughput info at a configurable period and allows writing to many files at once as well as using either mmap or write. This is a great test because it gives a bound on fs performance without any Kafka code involved. Added a test program that drives the LogManager level. Can write to any number of partitions and controls various configs. Add three test programs to test read and write locking in the OS. These are a bit hacky and I am fine leaving them on my machine too though they may come in handy again. (TestFileChannelLocking.scala, TestFileChannelReadLocking.scala, TestMmapLocking.scala) Misc things Remove the reference to flushInterval in many tests. Lots of tests were setting this to 1 to commit messages immediately. That is no longer necessary in 0.8. Added a new configuration replica.highwatermark.checkpoint.ms to control the checkpointing of hwm. Previously we used the config for log sync, but that doesn't make sense. Added Log.toString method for convenience Added explicit check for maxSize < 0 and startOffset > endOffset in LogSegment.read
        Hide
        jkreps Jay Kreps added a comment -

        Oops, wrong base revision for that patch, fixed it in v2.

        Show
        jkreps Jay Kreps added a comment - Oops, wrong base revision for that patch, fixed it in v2.
        Hide
        jkreps Jay Kreps added a comment -

        One more tweak.

        Show
        jkreps Jay Kreps added a comment - One more tweak.
        Hide
        nehanarkhede Neha Narkhede added a comment -

        +1.. its great to have this test checked in !

        Show
        nehanarkhede Neha Narkhede added a comment - +1.. its great to have this test checked in !

          People

          • Assignee:
            jkreps Jay Kreps
            Reporter:
            jkreps Jay Kreps
          • Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development