[ 
https://issues.apache.org/jira/browse/KAFKA-16284?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Matthias J. Sax updated KAFKA-16284:
------------------------------------
    Affects Version/s: 3.8.0

> Performance regression in RocksDB
> ---------------------------------
>
>                 Key: KAFKA-16284
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16284
>             Project: Kafka
>          Issue Type: Task
>          Components: streams
>    Affects Versions: 3.8.0
>            Reporter: Lucas Brutschy
>            Assignee: Lucas Brutschy
>            Priority: Major
>
> In benchmarks, we are noticing a performance regression in the performance of 
> `RocksDBStore`.
> The regression happens between those two commits:
>  
> {code:java}
> trunk - 70c8b8d0af - regressed - 2024-01-06T14:00:20Z
> trunk - d5aa341a18 - not regressed - 2023-12-31T11:47:14Z
> {code}
> The regression can be reproduced by the following test:
>  
> {code:java}
> package org.apache.kafka.streams.state.internals;
> import org.apache.kafka.common.serialization.Serdes;
> import org.apache.kafka.common.utils.Bytes;
> import org.apache.kafka.streams.StreamsConfig;
> import org.apache.kafka.streams.processor.StateStoreContext;
> import org.apache.kafka.test.InternalMockProcessorContext;
> import org.apache.kafka.test.MockRocksDbConfigSetter;
> import org.apache.kafka.test.StreamsTestUtils;
> import org.apache.kafka.test.TestUtils;
> import org.junit.Before;
> import org.junit.Test;
> import java.io.File;
> import java.nio.ByteBuffer;
> import java.util.Properties;
> public class RocksDBStorePerfTest {
>     InternalMockProcessorContext context;
>     RocksDBStore rocksDBStore;
>     final static String DB_NAME = "db-name";
>     final static String METRICS_SCOPE = "metrics-scope";
>     RocksDBStore getRocksDBStore() {
>         return new RocksDBStore(DB_NAME, METRICS_SCOPE);
>     }
>     @Before
>     public void setUp() {
>         final Properties props = StreamsTestUtils.getStreamsConfig();
>         props.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, 
> MockRocksDbConfigSetter.class);
>         File dir = TestUtils.tempDirectory();
>         context = new InternalMockProcessorContext<>(
>             dir,
>             Serdes.String(),
>             Serdes.String(),
>             new StreamsConfig(props)
>         );
>     }
>     @Test
>     public void testPerf() {
>         long start = System.currentTimeMillis();
>         for (int i = 0; i < 10; i++) {
>             System.out.println("Iteration: "+i+" Time: " + 
> (System.currentTimeMillis() - start));
>             RocksDBStore rocksDBStore = getRocksDBStore();
>             rocksDBStore.init((StateStoreContext) context, rocksDBStore);
>             for (int j = 0; j < 1000000; j++) {
>                 rocksDBStore.put(new 
> Bytes(ByteBuffer.allocate(4).putInt(j).array()), "perf".getBytes());
>             }
>             rocksDBStore.close();
>         }
>         long end = System.currentTimeMillis();
>         System.out.println("Time: " + (end - start));
>     }
> }
>  {code}
>  
> I have isolated the regression to commit 
> [5bc3aa4|https://github.com/apache/kafka/commit/5bc3aa428067dff1f2b9075ff5d1351fb05d4b10].
>  On my machine, the test takes ~8 seconds before 
> [5bc3aa4|https://github.com/apache/kafka/commit/5bc3aa428067dff1f2b9075ff5d1351fb05d4b10]
>  and ~30 seconds after 
> [5bc3aa4|https://github.com/apache/kafka/commit/5bc3aa428067dff1f2b9075ff5d1351fb05d4b10].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to