You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Sophie Blee-Goldman (Jira)" <ji...@apache.org> on 2019/11/06 01:18:00 UTC

[jira] [Created] (KAFKA-9148) Consider forking RocksDB for Streams

Sophie Blee-Goldman created KAFKA-9148:
------------------------------------------

             Summary: Consider forking RocksDB for Streams 
                 Key: KAFKA-9148
                 URL: https://issues.apache.org/jira/browse/KAFKA-9148
             Project: Kafka
          Issue Type: Improvement
          Components: streams
            Reporter: Sophie Blee-Goldman


We recently upgraded our RocksDB dependency to 5.18 for its memory-management abilities (WriteBufferManager -- KAFKA-8215). Unfortunately, someone recently discovered a ~8% performance regression that exists in all versions 5.18+ (latest being 6.2.2 as of now). Flink was able to react to this by downgrading to 5.17 and picking the WriteBufferManager to their fork, FRocksDB.

Due to this and other reasons enumerated below, we should consider also forking our own RocksDB for Streams.

 

Pros:
 * We can avoid passing sudden breaking changes on to our users, such removal of methods with no deprecation period (see discussion on KAFKA-8897)
 * We can pick whichever version has the best performance for our needs, and pick over any new features, metrics, etc that we need to use rather than being forced to upgrade (and breaking user code, introducing regression, etc)
 * The Java API seems to be a very low priority to the rocksdb folks.
 ** They leave out critical functionality, features, and configuration options that have been in the c++ API for a very long time
 ** Those that do make it over often have random gaps in the API such as setters but no getters (see [rocksdb PR #5186|https://github.com/facebook/rocksdb/pull/5186])
 ** Others are poorly designed and require too many trips across the JNI, making otherwise incredibly useful features prohibitively expensive.
 *** [Custom comparator|[https://github.com/facebook/rocksdb/issues/538#issuecomment-83145980]]: a custom comparator could significantly improve the performance of session windows
 *** [Prefix Seek|[https://github.com/facebook/rocksdb/issues/6004]]: not currently used by Streams but a commonly requested feature, and may also allow improved range queries
 ** Even when an external contributor develops a solution for poorly performing Java functionality and helpfully tries to contribute their patch back to rocksdb, it gets ignored by the rocksdb people ([rocksdb PR #2283|https://github.com/facebook/rocksdb/pull/2283])


Cons:
 * more work

 

Given that we rarely upgrade the Rocks dependency, use only some fraction of its features, and would need or want to make only minimal changes ourselves, it seems like we could actually get away with very little extra work by forking rocksdb. 

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Re: [jira] [Created] (KAFKA-9148) Consider forking RocksDB for Streams

Posted by Anbazhagan Muthuramalingam <an...@gmail.com>.
Unsubscribe

On Wed, Nov 6, 2019 at 6:48 AM Sophie Blee-Goldman (Jira) <ji...@apache.org>
wrote:

> Sophie Blee-Goldman created KAFKA-9148:
> ------------------------------------------
>
>              Summary: Consider forking RocksDB for Streams
>                  Key: KAFKA-9148
>                  URL: https://issues.apache.org/jira/browse/KAFKA-9148
>              Project: Kafka
>           Issue Type: Improvement
>           Components: streams
>             Reporter: Sophie Blee-Goldman
>
>
> We recently upgraded our RocksDB dependency to 5.18 for its
> memory-management abilities (WriteBufferManager -- KAFKA-8215).
> Unfortunately, someone recently discovered a ~8% performance regression
> that exists in all versions 5.18+ (latest being 6.2.2 as of now). Flink was
> able to react to this by downgrading to 5.17 and picking the
> WriteBufferManager to their fork, FRocksDB.
>
> Due to this and other reasons enumerated below, we should consider also
> forking our own RocksDB for Streams.
>
>
>
> Pros:
>  * We can avoid passing sudden breaking changes on to our users, such
> removal of methods with no deprecation period (see discussion on KAFKA-8897)
>  * We can pick whichever version has the best performance for our needs,
> and pick over any new features, metrics, etc that we need to use rather
> than being forced to upgrade (and breaking user code, introducing
> regression, etc)
>  * The Java API seems to be a very low priority to the rocksdb folks.
>  ** They leave out critical functionality, features, and configuration
> options that have been in the c++ API for a very long time
>  ** Those that do make it over often have random gaps in the API such as
> setters but no getters (see [rocksdb PR #5186|
> https://github.com/facebook/rocksdb/pull/5186])
>  ** Others are poorly designed and require too many trips across the JNI,
> making otherwise incredibly useful features prohibitively expensive.
>  *** [Custom comparator|[
> https://github.com/facebook/rocksdb/issues/538#issuecomment-83145980]]: a
> custom comparator could significantly improve the performance of session
> windows
>  *** [Prefix Seek|[https://github.com/facebook/rocksdb/issues/6004]]: not
> currently used by Streams but a commonly requested feature, and may also
> allow improved range queries
>  ** Even when an external contributor develops a solution for poorly
> performing Java functionality and helpfully tries to contribute their patch
> back to rocksdb, it gets ignored by the rocksdb people ([rocksdb PR #2283|
> https://github.com/facebook/rocksdb/pull/2283])
>
>
> Cons:
>  * more work
>
>
>
> Given that we rarely upgrade the Rocks dependency, use only some fraction
> of its features, and would need or want to make only minimal changes
> ourselves, it seems like we could actually get away with very little extra
> work by forking rocksdb.
>
>
>
>
>
> --
> This message was sent by Atlassian Jira
> (v8.3.4#803005)
>