[
https://issues.apache.org/jira/browse/KAFKA-2977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15063863#comment-15063863
]
ASF GitHub Bot commented on KAFKA-2977:
---------------------------------------
GitHub user ZoneMayor reopened a pull request:
https://github.com/apache/kafka/pull/671
KAFKA-2977: Transient Failure in
kafka.log.LogCleanerIntegrationTest.cleanerTest
Hi @guozhangwang
Code is as below:
val appends = writeDups(numKeys = 100, numDups = 3, log,
CompressionCodec.getCompressionCodec(compressionCodec))
cleaner.startup()
val firstDirty = log.activeSegment.baseOffset
cleaner.awaitCleaned("log", 0, firstDirty)
....
val appends2 = appends ++ writeDups(numKeys = 100, numDups = 3, log,
CompressionCodec.getCompressionCodec(compressionCodec))
val firstDirty2 = log.activeSegment.baseOffset
cleaner.awaitCleaned("log", 0, firstDirty2)
log cleaner and writeDups are two different threads;
log cleaner do cleaning every 15s, timeout in "cleaner.awaitCleaned" is 60s;
there is a filtering condition for a log to be chosen to become a cleaning
target: cleanableRatio> 0.5(configured log.cleaner.min.cleanable.ratio) by
default;
It may happen that, during "val appends2 = appends ++ writeDups(numKeys =
100, numDups = 3, log,
CompressionCodec.getCompressionCodec(compressionCodec))", log is also
undergoing a cleaning process;
Since the segment size configured in this test is quite small: 100, there
is possibility that before the end of 'writeDups', some 'dirty segment' of the
log is already cleaned;
With tiny dirty part left, cleanableRatio> 0.5 cannot be satisfied;
thus firstDirty2>lastCleaned2, which leads to this test failed;
Does it make sense?
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/ZoneMayor/kafka trunk-KAFKA-2977
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/kafka/pull/671.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 #671
----
commit 95374147a28208d4850f6e73f714bf418935fc2d
Author: ZoneMayor <[email protected]>
Date: 2015-11-27T03:49:34Z
Merge pull request #1 from apache/trunk
merge
commit cec5b48b651a7efd3900cfa3c1fd0ab1eeeaa3ec
Author: ZoneMayor <[email protected]>
Date: 2015-12-01T10:44:02Z
Merge pull request #2 from apache/trunk
2015-12-1
commit a119d547bf1741625ce0627073c7909992a20f15
Author: ZoneMayor <[email protected]>
Date: 2015-12-04T13:42:27Z
Merge pull request #3 from apache/trunk
2015-12-04#KAFKA-2893
commit b767a8dff85fc71c75d4cf5178c3f6f03ff81bfc
Author: ZoneMayor <[email protected]>
Date: 2015-12-09T10:42:30Z
Merge pull request #5 from apache/trunk
2015-12-9
commit 0070c2d71d06ee8baa1cddb3451cd5af6c6b1d4a
Author: ZoneMayor <[email protected]>
Date: 2015-12-11T14:50:30Z
Merge pull request #8 from apache/trunk
2015-12-11
commit 09908ac646d4c84f854dad63b8c99213b74a7063
Author: ZoneMayor <[email protected]>
Date: 2015-12-13T14:17:19Z
Merge pull request #9 from apache/trunk
2015-12-13
commit ff1e68bb7101d12624c189174ef1dceb21ed9798
Author: jinxing <[email protected]>
Date: 2015-12-13T14:31:34Z
KAFKA-2054: Transient Failure in
kafka.log.LogCleanerIntegrationTest.cleanerTest
commit 6321ab6599cb7a981fac2a4eea64a5f2ea805dd6
Author: jinxing <[email protected]>
Date: 2015-12-13T14:36:11Z
removed unnecessary maven repo
commit 05cae52c72a02c0ed40fd4e3be03e1cb19f33f7a
Author: jinxing <[email protected]>
Date: 2015-12-17T12:21:12Z
removed the semicolon
commit 651de48663cf375ea714cdbeb34650d75f1f4d71
Author: jinxing <[email protected]>
Date: 2015-12-18T07:43:38Z
KAFKA-2977: Transient Failure in
kafka.log.LogCleanerIntegrationTest.cleanerTest
commit ba03eee18045dc4aabc56ff17907036c238b1f7d
Author: jinxing <[email protected]>
Date: 2015-12-18T07:50:57Z
KAFKA-2977: fix
----
> Transient Failure in kafka.log.LogCleanerIntegrationTest.cleanerTest
> --------------------------------------------------------------------
>
> Key: KAFKA-2977
> URL: https://issues.apache.org/jira/browse/KAFKA-2977
> Project: Kafka
> Issue Type: Sub-task
> Affects Versions: 0.9.0.0
> Reporter: Guozhang Wang
> Assignee: jin xing
> Labels: newbie
> Fix For: 0.9.0.1
>
>
> {code}
> java.lang.AssertionError: log cleaner should have processed up to offset 599
> at org.junit.Assert.fail(Assert.java:88)
> at org.junit.Assert.assertTrue(Assert.java:41)
> at
> kafka.log.LogCleanerIntegrationTest.cleanerTest(LogCleanerIntegrationTest.scala:76)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
> at
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> at
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
> at
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> at
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
> at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at org.junit.runners.Suite.runChild(Suite.java:128)
> at org.junit.runners.Suite.runChild(Suite.java:27)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.runTestClass(JUnitTestClassExecuter.java:105)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.execute(JUnitTestClassExecuter.java:56)
> at
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassProcessor.processTestClass(JUnitTestClassProcessor.java:64)
> at
> org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:50)
> at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> at
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> at
> org.gradle.messaging.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
> at
> org.gradle.messaging.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
> at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
> at
> org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:106)
> at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> at
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> at
> org.gradle.messaging.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:360)
> at
> org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
> at
> org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
> 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:744)
> {code}
> Example: https://builds.apache.org/job/kafka-trunk-git-pr-jdk7/1699/
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)