-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: add KsqlRocksDBConfigSetter to bound memory and set num threads (…
- Loading branch information
Showing
10 changed files
with
770 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
46 changes: 46 additions & 0 deletions
46
ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/RocksDBConfigSetterHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* Copyright 2019 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.rest.util; | ||
|
||
import io.confluent.ksql.util.KsqlConfig; | ||
import java.util.Map; | ||
import org.apache.kafka.common.config.ConfigException; | ||
import org.apache.kafka.common.utils.Utils; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
|
||
public final class RocksDBConfigSetterHandler { | ||
|
||
private RocksDBConfigSetterHandler() { | ||
} | ||
|
||
public static void maybeConfigureRocksDBConfigSetter(final KsqlConfig ksqlConfig) { | ||
final Map<String, Object> streamsProps = ksqlConfig.getKsqlStreamConfigProps(); | ||
final Class<?> clazz = | ||
(Class) streamsProps.get(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG); | ||
|
||
if (clazz != null && org.apache.kafka.common.Configurable.class.isAssignableFrom(clazz)) { | ||
try { | ||
((org.apache.kafka.common.Configurable) Utils.newInstance(clazz)) | ||
.configure(ksqlConfig.originals()); | ||
} catch (Exception e) { | ||
throw new ConfigException( | ||
"Failed to configure Configurable RocksDBConfigSetter. " | ||
+ StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG + ": " + clazz.getName(), | ||
e); | ||
} | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
134 changes: 134 additions & 0 deletions
134
ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/RocksDBConfigSetterHandlerTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,134 @@ | ||
/* | ||
* Copyright 2019 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.rest.util; | ||
|
||
import static org.hamcrest.Matchers.containsString; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.when; | ||
|
||
import com.google.common.collect.ImmutableMap; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import io.confluent.ksql.util.KsqlException; | ||
import java.util.Map; | ||
import org.apache.kafka.common.config.ConfigException; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
import org.apache.kafka.streams.state.RocksDBConfigSetter; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.rules.ExpectedException; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
import org.rocksdb.Options; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class RocksDBConfigSetterHandlerTest { | ||
|
||
@Mock | ||
private KsqlConfig ksqlConfig; | ||
|
||
@Rule | ||
public final ExpectedException expectedException = ExpectedException.none(); | ||
|
||
@Test | ||
public void shouldConfigure() throws Exception { | ||
// Given: | ||
when(ksqlConfig.getKsqlStreamConfigProps()).thenReturn( | ||
ImmutableMap.of( | ||
StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, | ||
Class.forName("io.confluent.ksql.rest.util.RocksDBConfigSetterHandlerTest$ConfigurableTestRocksDBConfigSetter")) | ||
); | ||
final Runnable mockRunnable = mock(Runnable.class); | ||
when(ksqlConfig.originals()).thenReturn( | ||
ImmutableMap.of(ConfigurableTestRocksDBConfigSetter.TEST_CONFIG, mockRunnable)); | ||
|
||
// When: | ||
RocksDBConfigSetterHandler.maybeConfigureRocksDBConfigSetter(ksqlConfig); | ||
|
||
// Then: | ||
verify(mockRunnable).run(); | ||
} | ||
|
||
@Test | ||
public void shouldStartWithNonConfigurableRocksDBConfigSetter() throws Exception { | ||
// Given: | ||
when(ksqlConfig.getKsqlStreamConfigProps()).thenReturn( | ||
ImmutableMap.of( | ||
StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, | ||
Class.forName("io.confluent.ksql.rest.util.RocksDBConfigSetterHandlerTest$NonConfigurableTestRocksDBConfigSetter")) | ||
); | ||
|
||
// No error when: | ||
RocksDBConfigSetterHandler.maybeConfigureRocksDBConfigSetter(ksqlConfig); | ||
} | ||
|
||
@Test | ||
public void shouldThrowIfFailToRocksDBConfigSetter() throws Exception { | ||
// Given: | ||
when(ksqlConfig.getKsqlStreamConfigProps()).thenReturn( | ||
ImmutableMap.of( | ||
StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, | ||
Class.forName("io.confluent.ksql.rest.util.RocksDBConfigSetterHandlerTest$ConfigurableTestRocksDBConfigSetterWithoutPublicConstructor")) | ||
); | ||
|
||
// Expect: | ||
expectedException.expect(ConfigException.class); | ||
expectedException.expectMessage(containsString("Failed to configure Configurable RocksDBConfigSetter.")); | ||
expectedException.expectMessage(containsString(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG)); | ||
expectedException.expectMessage(containsString("io.confluent.ksql.rest.util.RocksDBConfigSetterHandlerTest$ConfigurableTestRocksDBConfigSetterWithoutPublicConstructor")); | ||
|
||
// When: | ||
RocksDBConfigSetterHandler.maybeConfigureRocksDBConfigSetter(ksqlConfig); | ||
} | ||
|
||
public static class ConfigurableTestRocksDBConfigSetter | ||
extends NonConfigurableTestRocksDBConfigSetter | ||
implements org.apache.kafka.common.Configurable { | ||
|
||
static final String TEST_CONFIG = "test.runnable"; | ||
|
||
@Override | ||
public void configure(final Map<String, ?> config) { | ||
final Runnable supplier = (Runnable) config.get(TEST_CONFIG); | ||
supplier.run(); | ||
} | ||
} | ||
|
||
static class ConfigurableTestRocksDBConfigSetterWithoutPublicConstructor | ||
extends NonConfigurableTestRocksDBConfigSetter | ||
implements org.apache.kafka.common.Configurable { | ||
|
||
@Override | ||
public void configure(final Map<String, ?> config) { | ||
} | ||
} | ||
|
||
private static class NonConfigurableTestRocksDBConfigSetter implements RocksDBConfigSetter { | ||
|
||
@Override | ||
public void setConfig( | ||
final String storeName, | ||
final Options options, | ||
final Map<String, Object> configs) { | ||
// do nothing | ||
} | ||
|
||
@Override | ||
public void close(final String storeName, final Options options) { | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
~ Copyright 2019 Confluent Inc. | ||
~ | ||
~ Licensed under the Confluent Community License (the "License"); you may not use | ||
~ this file except in compliance with the License. You may obtain a copy of the | ||
~ License at | ||
~ | ||
~ http://www.confluent.io/confluent-community-license | ||
~ | ||
~ Unless required by applicable law or agreed to in writing, software | ||
~ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
~ WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
~ specific language governing permissions and limitations under the License. | ||
--> | ||
|
||
<project xmlns="http://maven.apache.org/POM/4.0.0" | ||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<parent> | ||
<groupId>io.confluent.ksql</groupId> | ||
<artifactId>ksql-parent</artifactId> | ||
<version>5.3.1-SNAPSHOT</version> | ||
</parent> | ||
|
||
<artifactId>ksql-rocksdb-config-setter</artifactId> | ||
<name>KSQL RocksDB Config Setter</name> | ||
<packaging>jar</packaging> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.kafka</groupId> | ||
<artifactId>kafka-streams</artifactId> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>junit</groupId> | ||
<artifactId>junit</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.hamcrest</groupId> | ||
<artifactId>hamcrest-all</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.mockito</groupId> | ||
<artifactId>mockito-core</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.google.guava</groupId> | ||
<artifactId>guava</artifactId> | ||
</dependency> | ||
</dependencies> | ||
|
||
|
||
|
||
<build> | ||
<plugins> | ||
<plugin> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-jar-plugin</artifactId> | ||
<executions> | ||
<execution> | ||
<goals> | ||
<goal>test-jar</goal> | ||
</goals> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> |
Oops, something went wrong.