Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-16284

Performance regression in RocksDB

    XMLWordPrintableJSON

Details

    • Task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • None
    • streams
    • None

    Description

      In benchmarks, we are noticing a performance regression in the performance of `RocksDBStore`.

      The regression happens between those two commits:
       

      trunk - 70c8b8d0af - regressed - 2024-01-06T14:00:20Z
      trunk - d5aa341a18 - not regressed - 2023-12-31T11:47:14Z
      

      The regression can be reproduced by the following test:
       

      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));
          }
      
      }
       

       
      I have isolated the regression to commit 5bc3aa4. On my machine, the test takes ~8 seconds before 5bc3aa4 and ~30 seconds after 5bc3aa4.

      Attachments

        Issue Links

          Activity

            People

              lucasbru Lucas Brutschy
              lucasbru Lucas Brutschy
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: