[ https://issues.apache.org/jira/browse/KAFKA-3502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15823052#comment-15823052 ]
ASF GitHub Bot commented on KAFKA-3502: --------------------------------------- GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2381 KAFKA-3502: move RocksDB options construction to init() In RocksDBStore, options / wOptions / fOptions are constructed in the constructor, which needs to be dismissed in the close() call; however in some tests, the generated topology is not initialized at all, and hence the corresponding state stores are supposed to not be able to be closed as well since their `init` function is not called. This could cause the above option objects to be not released. This is fixed in this patch to move the logic out of constructor and inside `init` functions, so that no RocksDB objects will be created in the constructor only. Also some minor cleanups: 1. In KStreamTestDriver.close(), we lost the logic to close the state stores but only call `flush`; it is now changed back to call both. 2. Moved the forwarding logic from KStreamTestDriver to MockProcessorContext to remove the mutual dependency: these functions should really be in ProcessorContext, not the test driver. You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K3502-pure-virtual-function-unit-tests Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/2381.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 #2381 ---- commit 78870f73fbdc6c4e708b5ebe9d9c4c1e495e5a5d Author: Guozhang Wang <wangg...@gmail.com> Date: 2017-01-15T06:22:37Z move options construction to init() ---- > Build is killed during kafka streams tests due to `pure virtual method > called` error > ------------------------------------------------------------------------------------ > > Key: KAFKA-3502 > URL: https://issues.apache.org/jira/browse/KAFKA-3502 > Project: Kafka > Issue Type: Sub-task > Reporter: Ashish K Singh > Assignee: Guozhang Wang > Labels: transient-unit-test-failure > > Build failed due to failure in streams' test. Not clear which test led to > this. > Jenkins console: > https://builds.apache.org/job/kafka-trunk-git-pr-jdk7/3210/console > {code} > org.apache.kafka.streams.kstream.internals.KTableFilterTest > testValueGetter > PASSED > org.apache.kafka.streams.kstream.internals.KStreamFlatMapTest > testFlatMap > PASSED > org.apache.kafka.streams.kstream.internals.KTableAggregateTest > testAggBasic > PASSED > org.apache.kafka.streams.kstream.internals.KStreamFlatMapValuesTest > > testFlatMapValues PASSED > org.apache.kafka.streams.kstream.KStreamBuilderTest > testMerge PASSED > org.apache.kafka.streams.kstream.KStreamBuilderTest > testFrom PASSED > org.apache.kafka.streams.kstream.KStreamBuilderTest > testNewName PASSED > pure virtual method called > terminate called without an active exception > :streams:test FAILED > FAILURE: Build failed with an exception. > * What went wrong: > Execution failed for task ':streams:test'. > > Process 'Gradle Test Executor 4' finished with non-zero exit value 134 > {code} > Tried reproducing the issue locally, but could not. -- This message was sent by Atlassian JIRA (v6.3.4#6332)