Skip to content

Commit

Permalink
[hotfix] Remove unnecessary job id from RocksDBKeyedStateBackend
Browse files Browse the repository at this point in the history
  • Loading branch information
tillrohrmann committed May 18, 2017
1 parent f2af1a9 commit 20b356c
Show file tree
Hide file tree
Showing 3 changed files with 4 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.flink.contrib.streaming.state;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.state.AggregatingStateDescriptor;
import org.apache.flink.api.common.state.FoldingStateDescriptor;
import org.apache.flink.api.common.state.ListStateDescriptor;
Expand Down Expand Up @@ -180,7 +179,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
private static final String SST_FILE_SUFFIX = ".sst";

public RocksDBKeyedStateBackend(
JobID jobId,
String operatorIdentifier,
ClassLoader userCodeClassLoader,
File instanceBasePath,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,10 +300,9 @@ public <K> AbstractKeyedStateBackend<K> createKeyedStateBackend(
lazyInitializeForJob(env, operatorIdentifier);

File instanceBasePath =
new File(getNextStoragePath(), "job-" + jobId.toString() + "_op-" + operatorIdentifier + "_uuid-" + UUID.randomUUID());
new File(getNextStoragePath(), "job-" + jobId + "_op-" + operatorIdentifier + "_uuid-" + UUID.randomUUID());

return new RocksDBKeyedStateBackend<>(
jobID,
operatorIdentifier,
env.getUserClassLoader(),
instanceBasePath,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.test.query;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
Expand Down Expand Up @@ -66,7 +65,6 @@ public final class KVStateRequestSerializerRocksDBTest {
final static class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {

RocksDBKeyedStateBackend2(
final JobID jobId,
final String operatorIdentifier,
final ClassLoader userCodeClassLoader,
final File instanceBasePath,
Expand All @@ -78,7 +76,7 @@ final static class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend
final KeyGroupRange keyGroupRange,
final ExecutionConfig executionConfig) throws Exception {

super(jobId, operatorIdentifier, userCodeClassLoader,
super(operatorIdentifier, userCodeClassLoader,
instanceBasePath,
dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer,
numberOfKeyGroups, keyGroupRange, executionConfig, false);
Expand Down Expand Up @@ -110,7 +108,7 @@ public void testListSerialization() throws Exception {
ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
final RocksDBKeyedStateBackend2<Long> longHeapKeyedStateBackend =
new RocksDBKeyedStateBackend2<>(
new JobID(), "no-op",
"no-op",
ClassLoader.getSystemClassLoader(),
temporaryFolder.getRoot(),
dbOptions,
Expand Down Expand Up @@ -147,7 +145,7 @@ public void testMapSerialization() throws Exception {
ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
final RocksDBKeyedStateBackend<Long> longHeapKeyedStateBackend =
new RocksDBKeyedStateBackend<>(
new JobID(), "no-op",
"no-op",
ClassLoader.getSystemClassLoader(),
temporaryFolder.getRoot(),
dbOptions,
Expand Down

0 comments on commit 20b356c

Please sign in to comment.