You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/01/17 18:51:28 UTC

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

    [ https://issues.apache.org/jira/browse/KAFKA-1194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826590#comment-15826590 ] 

ASF GitHub Bot commented on KAFKA-1194:
---------------------------------------

GitHub user silpamittapalli opened a pull request:

    https://github.com/apache/kafka/pull/2386

    Upgrade Kafka and apply windows fix

    Based on https://github.com/fluetm/kafka/tree/kafka-upgrade.
    
    1. Cherry-picked windows fix for https://issues.apache.org/jira/browse/KAFKA-1194 from https://github.com/fluetm/kafka/tree/1194-fix
    2. Resolved merge conflicts and customized the windows fix so that it fits in the released version of Kafka we upgraded to, rather than the unreleased trunk


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/silpamittapalli/kafka sm_kafka-windows

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/2386.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2386
    
----
commit f66e88a30b2aabd02bc0ef35fa27f8bd9b35466f
Author: Ben Stopford <be...@gmail.com>
Date:   2016-09-20T06:17:23Z

    KAFKA-4193; Fix for intermittent failure in FetcherTest
    
    Author: Ben Stopford <be...@gmail.com>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1881 from benstopford/KAFKA-4193
    
    (cherry picked from commit f396fdac197409fb955f00a6f642f04e4926ba41)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit fc5f48aad4952df88147675a663ad034ce15d13d
Author: Eno Thereska <en...@gmail.com>
Date:   2016-09-20T10:33:50Z

    HOTFIX: Added check for metadata unavailable
    
    Author: Eno Thereska <en...@gmail.com>
    
    Reviewers: Damian Guy <da...@gmail.com>, Ismael Juma <is...@juma.me.uk>
    
    Closes #1887 from enothereska/hotfix-metadata-unavailable

commit d48415f185d1882f0a3b89a3ce03ea84893393ba
Author: Ben Stopford <be...@gmail.com>
Date:   2016-09-20T13:53:48Z

    KAFKA-4184; Intermittent failures in ReplicationQuotasTest.shouldBootstrapTwoBrokersWithFollowerThrottle
    
    Build is unstable, so it's hard to validate this change. Of the various builds up until 11am BST the test ran twice and passed twice.
    
    Author: Ben Stopford <be...@gmail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1873 from benstopford/KAFKA-4184
    
    (cherry picked from commit 3663275cf066b7715cc11b26fd9c144bbff1c373)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit 29e30a79e5ba4f137d77943976b1e8e77f6ccaac
Author: Ben Stopford <be...@gmail.com>
Date:   2016-09-20T14:41:14Z

    KAFKA-4197; Make ReassignPartitionsTest System Test move data
    
    The ReassignPartitionsTest system tests doesn't reassign any replicas (i.e. move data).
    
    This is a simple issue. It uses a 3 node cluster with replication factor of 3, so whilst the replicas are jumbled around, nothing actually is moved from machine to machine when the assignment is executed.
    
    This fix just ups the number of nodes to 4 so things move.
    
    Tests pass locally.
    There are runs pending on the two branch builders
    
    Passes:
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder/551/
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder-2/94/
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder/553/
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder/554/
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder-2/95
    
    Failures:
    https://jenkins.confluent.io/job/system-test-kafka-branch-builder/552 => _RuntimeError: There aren't enough available nodes to satisfy the resource request. Total cluster size: 1, Requested: 4, Already allocated: 1, Available: 0._ Which I assume to do with the test env.
    
    Author: Ben Stopford <be...@gmail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1892 from benstopford/fix_reassignment_test
    
    (cherry picked from commit 4f821830bc6b726cddf90999fff76006745b1a3f)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit 5b29bb8b039c96634d82352c7e17922c83dad48f
Author: Damian Guy <da...@gmail.com>
Date:   2016-09-21T18:11:12Z

    MINOR: add javadoc comment to PersistenKeyValueFactory.enableCaching
    
    missing javadoc on public API method PersistenKeyValueFactory.enableCaching
    
    Author: Damian Guy <da...@gmail.com>
    
    Reviewers: Eno Thereska, Guozhang Wang
    
    Closes #1891 from dguy/minor-java-doc
    
    (cherry picked from commit 24f81ea764a493b4422b6a3ef6b3e771d0e4d63b)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit be20ea52892c91f59323e0be1108f689e5a44f95
Author: Damian Guy <da...@gmail.com>
Date:   2016-09-21T18:13:39Z

    MINOR: remove unused code from InternalTopicManager
    
    Remove isValidCleanupPolicy and related fields as they are never used.
    
    Author: Damian Guy <da...@gmail.com>
    
    Reviewers: Eno Thereska, Guozhang Wang
    
    Closes #1888 from dguy/minor-remove-unused
    
    (cherry picked from commit a632716a3c9a871f325c6f13aefa9aed0add4b82)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit c9f7fa29d1b0c165f13ff6e18442580b04cd8ec9
Author: Jason Gustafson <ja...@confluent.io>
Date:   2016-09-22T17:07:50Z

    KAFKA-3782: Ensure heartbeat thread restarted after rebalance woken up
    
    Author: Jason Gustafson <ja...@confluent.io>
    
    Reviewers: Guozhang Wang
    
    Closes #1898 from hachikuji/KAFKA-3782
    
    (cherry picked from commit 732fabf94ebc9631d31f2feb2116ee8b63beabef)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit d1c8592b319f0ed1d8a3f70bee2b8b994c22e694
Author: Elias Levy <fe...@gmail.com>
Date:   2016-09-22T17:33:23Z

    MINOR: Fix comments in KStreamKStreamJoinTest
    
    Minor comment fixes.
    
    Author: Elias Levy <fe...@gmail.com>
    
    Reviewers: Guozhang Wang
    
    Closes #1885 from eliaslevy/fix-test-comments
    
    (cherry picked from commit 27e3edc791760dea7ff4d048f87d1585f9e235d7)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit 5006afe81defaf6690fcdf05303465907ef40960
Author: Eno Thereska <en...@gmail.com>
Date:   2016-09-22T19:45:43Z

    HOTFIX: Decrease commit interval
    
    The original commit interval of 30 seconds might be too large in some cases, e.g., when the verifier finishes before those 30 seconds have elapsed.
    
    Author: Eno Thereska <en...@gmail.com>
    
    Reviewers: Damian Guy, Guozhang Wang
    
    Closes #1899 from enothereska/hotfix-smoke-test-commit-interval
    
    (cherry picked from commit 4112bc3af7bec63ae1248c1b9db745ba8ac4c63c)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit dc5fc239eef887c18db130ade65779b5821c737a
Author: Ismael Juma <is...@juma.me.uk>
Date:   2016-09-22T21:53:40Z

    MINOR: Increase `zkConnectionTimeout` and timeout in `testReachableServer`
    
    We had a number of failures recently due to these timeouts being too low. It's a particular problem if multiple forks are used while running the tests.
    
    Author: Ismael Juma <is...@juma.me.uk>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1889 from ijuma/increase-zk-timeout-in-tests
    
    (cherry picked from commit d32f3f2828c199a5a09bbba779d07430e43fb190)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit a859cedf0b9d3b91dd411623d769961d245cc7af
Author: Jason Gustafson <ja...@confluent.io>
Date:   2016-09-23T20:13:29Z

    KAFKA-3590; Handle not-enough-replicas errors when writing to offsets topic
    
    Author: Jason Gustafson <ja...@confluent.io>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>, Guozhang Wang <wa...@gmail.com>
    
    Closes #1859 from hachikuji/KAFKA-3590
    
    (cherry picked from commit 6a13a3dbaddf99850b2583007577fa2a6e1e6d3a)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit cd252352cf7aacaf3987c15a2edf3f3dd4751fac
Author: Ryan Pridgeon <ry...@gmail.com>
Date:   2016-09-23T21:20:41Z

    KAFKA-3719; Allow underscores in hostname
    
    Technically this does not strictly adhere to RFC-952 however it is valid for domain names, urls and uris so we should loosen the requirements a tad.
    
    Author: Ryan Pridgeon <ry...@gmail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1856 from rnpridgeon/KAFKA-3719
    
    (cherry picked from commit 0de807357be4d776747e960b5fd17ddeae9a6cad)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit 97f41a5c1193146f9c064a7c2c6265ee340036a5
Author: Sumit Arrawatia <su...@gmail.com>
Date:   2016-09-24T09:16:49Z

    KAFKA-4151; Update public docs for Cluster Id (KIP-78)
    
    - Updated implementation docs with details on Cluster Id generation.
    - Mention cluster id in "noteworthy changes for 0.10.1.0" in upgrade docs.
    
    Author: Sumit Arrawatia <su...@gmail.com>
    Author: arrawatia <su...@gmail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1895 from arrawatia/kip-78-docs
    
    (cherry picked from commit 36242b846a42b33d7d4c1931f2dae93ebe1547c7)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit e4d8059d7eddb3399ef3888c70a2c939ff599cca
Author: Arun Mahadevan <ai...@hortonworks.com>
Date:   2016-09-25T07:44:56Z

    KAFKA-3282; Change tools to use new consumer if zookeeper is not specified
    
    Author: Arun Mahadevan <ai...@hortonworks.com>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>, Ismael Juma <is...@juma.me.uk>
    
    Closes #1376 from arunmahadevan/cons-consumer-fix
    
    (cherry picked from commit 1d055f7551d138324d2540095a1cfc1c8f74d76f)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit 4104f014e92bbf6878e575a93a850ff3a569ccf0
Author: Rajini Sivaram <ra...@googlemail.com>
Date:   2016-09-26T00:01:45Z

    KAFKA-4055; System tests for secure quotas
    
    Fix existing client-id quota test which currently don't configure quota overrides correctly. Add new tests for user and (user, client-id) quota overrides and default quotas.
    
    Author: Rajini Sivaram <ra...@googlemail.com>
    
    Reviewers: Jun Rao <ju...@gmail.com>
    
    Closes #1860 from rajinisivaram/KAFKA-4055
    
    (cherry picked from commit c0a62b70a8eadc550c937bb18e0203ab691618f5)
    Signed-off-by: Jun Rao <ju...@gmail.com>

commit d6b3ff142a8ab19264b7af76bda7ddaa8c4bea1d
Author: Jason Gustafson <ja...@confluent.io>
Date:   2016-09-26T21:54:01Z

    MINOR: Wakeups propagated from commitOffsets in WorkerSinkTask should be caught
    
    Author: Jason Gustafson <ja...@confluent.io>
    
    Reviewers: Ewen Cheslack-Postava <ew...@confluent.io>
    
    Closes #1907 from hachikuji/catch-wakeup-worker-sink-task
    
    (cherry picked from commit b75245cfbbefc712103b9329da0f27a205baa6aa)
    Signed-off-by: Ewen Cheslack-Postava <me...@ewencp.org>

commit 823c08067c189791d97ca4bd0548ffa308684dd7
Author: Apurva Mehta <ap...@gmail.com>
Date:   2016-09-27T00:18:18Z

    KAFKA-4214; kafka-reassign-partitions fails all the time when brokers are bounced during reassignment
    
    There is a corner case bug, where during partition reassignment, if the
    controller and a broker receiving a new replica are bounced at the same
    time, the partition reassignment is failed.
    
    The cause of this bug is a block of code in the KafkaController which
    fails the reassignment if the aliveNewReplicas != newReplicas, ie. if
    some of the new replicas are offline at the time a controller fails
    over.
    
    The fix is to have the controller listen for ISR change events even for
    new replicas which are not alive when the controller boots up. Once the
    said replicas come online, they will be in the ISR set, and the new
    controller will detect this, and then mark the reassignment as
    successful.
    
    Interestingly, the block of code in question was introduced in
    KAFKA-990, where a concern about this exact scenario was raised :)
    
    This bug was revealed in the system tests in https://github.com/apache/kafka/pull/1904.
    The relevant tests will be enabled in either this or a followup PR when PR-1904 is merged.
    
    Thanks to junrao identifying the issue and providing the patch.
    
    Author: Apurva Mehta <ap...@gmail.com>
    
    Reviewers: Jun Rao <ju...@gmail.com>
    
    Closes #1910 from apurvam/KAFKA-4214
    
    (cherry picked from commit be6056abc9970600347c95c4c8658799b76dbe6b)
    Signed-off-by: Jun Rao <ju...@gmail.com>

commit 6b8bcf65e87a1fcdc32137e3fc877d3296bae074
Author: Vahid Hashemian <va...@us.ibm.com>
Date:   2016-09-27T02:04:22Z

    KAFKA-3831; Prepare for updating new-consumer-based Mirror Maker's default partition assignment strategy to round robin
    
    This patch adds proper warning message and necessary doc updates for updating the default partition assignment strategy of Mirror Maker from range to round robin. The actual switch would occur as part of a major release cycle (to be scheduled).
    
    Author: Vahid Hashemian <va...@us.ibm.com>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1499 from vahidhashemian/KAFKA-3831
    
    (cherry picked from commit 60ad6d727861a87fa756918a7be7547e9b1f4c3d)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit 98a625028994e84bafd1cf7d96233dfdb9579555
Author: Magnus Reftel <ma...@skatteetaten.no>
Date:   2016-09-27T04:15:25Z

    MINOR: Fix Javadoc of @throws AuthorizationException for KafkaConsumer.poll
    
    Author: Magnus Reftel <ma...@skatteetaten.no>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1901 from reftel/feature/poll_javadoc
    
    (cherry picked from commit a208478f37f216324a87b530f18f30ecd79601c8)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit 55631c976639c5c9295055553a6fe3bd06a66b21
Author: Ben Stopford <be...@gmail.com>
Date:   2016-09-27T13:00:44Z

    KAFKA-4200; Fix throttle argument in kafka-reassign-partitions.sh
    
    Simple jira which alters two things:
    
    1. kafka-reassign-partitions --verify prints Throttle was removed regardless of whether a throttle was applied. It should only print this if the value was actually changed.
    
    2. --verify should exception if the —throttle argument. (check generate too)
    
    To test this I extracted all validation logic into a separate method and added a test which covers the majority of combinations. The validation logic was retained as is, other than implementing (2) and adding validation to the --broker-list option which you can currently apply to any of hte main actions (where it is ignored). Requirement 1 was tested manually (as it's just println).
    
    Testing:
    - Build passes locally.
    - System test reassign_partitions_test.py also passes.
    
    Author: Ben Stopford <be...@gmail.com>
    
    Reviewers: Jun Rao <ju...@gmail.com>
    
    Closes #1896 from benstopford/KAFKA-4200
    
    (cherry picked from commit 5d6408f6cfda3f8ab366195f69e90de048cde25d)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit bb3a86087ca6ac089bf10382e5cacd6f85437315
Author: Ben Stopford <be...@gmail.com>
Date:   2016-09-27T19:27:20Z

    KAFKA-4177; Remove ThrottledReplicationRateLimit from Server Config
    
    This small PR pulls ThrottledReplicationRateLimit out of KafkaConfig and puts it in a class that defines Dynamic Configs. Client configs are also placed in this class and validation added.
    
    Author: Ben Stopford <be...@gmail.com>
    
    Reviewers: Jun Rao <ju...@gmail.com>, Ismael Juma <is...@juma.me.uk>
    
    Closes #1864 from benstopford/KAFKA-4177
    
    (cherry picked from commit b8ed4a51134ca8b98d1445871a55ed33b6ad5b92)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit aadda5aaceadf40d8dbebcdd577962485854f0cb
Author: Jason Gustafson <ja...@confluent.io>
Date:   2016-09-27T20:35:29Z

    MINOR: Make new consumer default for Mirror Maker
    
    Author: Jason Gustafson <ja...@confluent.io>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1914 from hachikuji/mm-default-new-consumer
    
    (cherry picked from commit 3db752a565071c78e4b11eaafa739844fa785b04)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit dfdf2e6cc026caef0dd02b2d870ef7bf86508b22
Author: Ismael Juma <is...@juma.me.uk>
Date:   2016-09-27T23:49:46Z

    MINOR: Remove no longer required --new-consumer switch in docs
    
    Author: Ismael Juma <is...@juma.me.uk>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1905 from ijuma/no-new-consumer-switch-in-examples
    
    (cherry picked from commit 61d3378bc84914a521a65cdfffb7299928fa8671)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit 65b85e6a490a1f121071da735e2abfce807c87d1
Author: Damian Guy <da...@gmail.com>
Date:   2016-09-28T00:35:24Z

    HOTFIX: fix npe in StreamsMetadataState when onChange has not been called
    
    If some StreamsMetadataState methods are called before the onChange method is called a NullPointerException was being thrown. Added null check for cluster in isInitialized method
    
    Author: Damian Guy <da...@gmail.com>
    
    Reviewers: Guozhang Wang <wa...@gmail.com>
    
    Closes #1920 from dguy/fix-npe-streamsmetadata
    
    (cherry picked from commit 0c25c73782e6e70b8f37e3dda2fa2a5b0b1c8c65)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit aef1c13a32404a4534d72ba30c5d7652391c1488
Author: Guozhang Wang <wa...@gmail.com>
Date:   2016-09-28T00:37:12Z

    MINOR: fixes a few error logging formats
    
    Author: Guozhang Wang <wa...@gmail.com>
    
    Reviewers: Eno Thereska <en...@gmail.com>
    
    Closes #1919 from guozhangwang/minor-error-message-fixes
    
    (cherry picked from commit 0c4cc5a4466d0dfce01903c29d97af3e428597f3)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit 1fcf9fd6287db7de1abd9be9a73f1fa8a4ecd402
Author: Damian Guy <da...@gmail.com>
Date:   2016-09-28T00:43:36Z

    MINOR: add test to make sure ProcessorStateManager can handle State Stores with logging disabled
    
    Adding the test so we know that the State Stores with logging disabled or without a topic don't throw any exceptions.
    
    Author: Damian Guy <da...@gmail.com>
    
    Reviewers: Guozhang Wang <wa...@gmail.com>
    
    Closes #1916 from dguy/state-store-logging-disabled
    
    (cherry picked from commit c526c0c3f6f153ef8f84a3e66d9f997d8c31e20c)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit 66b2caafe9f8eef4476558277401760ee9077480
Author: Damian Guy <da...@gmail.com>
Date:   2016-09-28T18:07:44Z

    KAFKA-3708: Better exception handling in Kafka Streams
    
    KafkaExceptions currently thrown from within StreamThread/StreamTask currently bubble up without any additional context. This makes it hard to figure out where something went wrong, i.e, which topic had the serialization exception etc
    
    Author: Damian Guy <da...@gmail.com>
    
    Reviewers: Guozhang Wang <wa...@gmail.com>
    
    Closes #1819 from dguy/kafka-3708 and squashes the following commits:
    
    d6feaa8 [Damian Guy] address comments
    15b89e7 [Damian Guy] merge trunk
    6b8a8af [Damian Guy] catch exceptions in various places and throw more informative versions
    c86eeda [Damian Guy] fix conflicts
    8f37e2c [Damian Guy] add some context to exceptions
    
    (cherry picked from commit d83cde7cabe4e86951c6760e68e65b99752cfe0e)
    Signed-off-by: Guozhang Wang <wa...@gmail.com>

commit 7e67d87c121414fa16465fa34309053c9531b9ac
Author: Rajini Sivaram <ra...@googlemail.com>
Date:   2016-09-28T19:18:20Z

    KAFKA-4227; Shutdown AdminManager when KafkaServer is shutdown
    
    Terminate topic purgatory thread in AdminManager during server shutdown to avoid threads being left around in unit tests.
    
    Author: Rajini Sivaram <ra...@googlemail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
    
    Closes #1927 from rajinisivaram/KAFKA-4227
    
    (cherry picked from commit 281fac9ed626b16190d56e609c9bf04a3188374a)
    Signed-off-by: Ismael Juma <is...@juma.me.uk>

commit 003c333f62d7edddcd12f8936fef3b22e27bd605
Author: Jiangjie Qin <be...@gmail.com>
Date:   2016-09-29T00:45:08Z

    KAFKA-4194; Follow-up improvements/testing for ListOffsets v1 (KIP-79)
    
    Author: Jiangjie Qin <be...@gmail.com>
    
    Reviewers: Ismael Juma <is...@juma.me.uk>, Jason Gustafson <ja...@confluent.io>
    
    Closes #1897 from becketqin/KAFKA-4194
    
    (cherry picked from commit aa506a6919e8ec0381657db2c2f15dc1f62b00f6)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

commit 2353f100fb7502ab2dc6c527485c3263f618df52
Author: Ismael Juma <is...@juma.me.uk>
Date:   2016-09-29T02:15:00Z

    MINOR: Set JVM parameters for the Gradle Test executor processes
    
    We suspect that the test suite hangs we have been seeing are
    due to PermGen exhaustion. It is a common reason for
    hard JVM lock-ups.
    
    Author: Ismael Juma <is...@juma.me.uk>
    
    Reviewers: Jason Gustafson <ja...@confluent.io>
    
    Closes #1926 from ijuma/test-jvm-params
    
    (cherry picked from commit 67e99d0869dd49358d7ca549ac715b722fda89f5)
    Signed-off-by: Jason Gustafson <ja...@confluent.io>

----


> The kafka broker cannot delete the old log files after the configured time
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-1194
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1194
>             Project: Kafka
>          Issue Type: Bug
>          Components: log
>    Affects Versions: 0.8.1
>         Environment: window
>            Reporter: Tao Qin
>            Assignee: Jay Kreps
>              Labels: features, patch
>             Fix For: 0.10.2.0
>
>         Attachments: KAFKA-1194.patch, kafka-1194-v1.patch, kafka-1194-v2.patch, screenshot-1.png, Untitled.jpg
>
>   Original Estimate: 72h
>  Remaining Estimate: 72h
>
> We tested it in windows environment, and set the log.retention.hours to 24 hours.
> # The minimum age of a log file to be eligible for deletion
> log.retention.hours=24
> After several days, the kafka broker still cannot delete the old log file. And we get the following exceptions:
> [2013-12-19 01:57:38,528] ERROR Uncaught exception in scheduled task 'kafka-log-retention' (kafka.utils.KafkaScheduler)
> kafka.common.KafkaStorageException: Failed to change the log file suffix from  to .deleted for log segment 1516723
>          at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:249)
>          at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:638)
>          at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:629)
>          at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:418)
>          at kafka.log.Log$$anonfun$deleteOldSegments$1.apply(Log.scala:418)
>          at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:59)
>          at scala.collection.immutable.List.foreach(List.scala:76)
>          at kafka.log.Log.deleteOldSegments(Log.scala:418)
>          at kafka.log.LogManager.kafka$log$LogManager$$cleanupExpiredSegments(LogManager.scala:284)
>          at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:316)
>          at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:314)
>          at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:743)
>          at scala.collection.Iterator$class.foreach(Iterator.scala:772)
>          at scala.collection.JavaConversions$JIteratorWrapper.foreach(JavaConversions.scala:573)
>          at scala.collection.IterableLike$class.foreach(IterableLike.scala:73)
>          at scala.collection.JavaConversions$JListWrapper.foreach(JavaConversions.scala:615)
>          at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:742)
>          at kafka.log.LogManager.cleanupLogs(LogManager.scala:314)
>          at kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:143)
>          at kafka.utils.KafkaScheduler$$anon$1.run(KafkaScheduler.scala:100)
>          at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>          at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>          at java.lang.Thread.run(Thread.java:724)
> I think this error happens because kafka tries to rename the log file when it is still opened.  So we should close the file first before rename.
> The index file uses a special data structure, the MappedByteBuffer. Javadoc describes it as:
> A mapped byte buffer and the file mapping that it represents remain valid until the buffer itself is garbage-collected.
> Fortunately, I find a forceUnmap function in kafka code, and perhaps it can be used to free the MappedByteBuffer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)