-
Notifications
You must be signed in to change notification settings - Fork 13k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-7475] [core][DataStream API] support update() in ListState #4963
Changes from 18 commits
e06ac7e
8df3cf1
25e9429
c01be2b
1ad59b3
b565478
a148b1a
7dfc24e
a430024
f8a5bda
793c056
5a73127
c37d926
831159e
9afa713
795ea1d
97d7cfa
ee00a8a
a1bde36
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.contrib.streaming.state.util; | ||
|
||
import org.apache.flink.annotation.VisibleForTesting; | ||
|
||
import java.util.List; | ||
|
||
/** | ||
* Utils to simulate StringAppendTestOperator's merge operations in RocksDB. | ||
*/ | ||
public class MergeUtils { | ||
@VisibleForTesting | ||
protected static final byte DELIMITER = ','; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It would be great if we could initialize this from the RocksDB API somehow. But not sure if this is possible. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. RocksDB doesn't expose that There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, that is unfortunate. |
||
|
||
/** | ||
* Merge operands into a single value that can be put directly into RocksDB. | ||
*/ | ||
public static byte[] merge(List<byte[]> operands) { | ||
if (operands == null || operands.size() == 0) { | ||
return null; | ||
} | ||
|
||
if (operands.size() == 1) { | ||
return operands.get(0); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This will fail if size == 0. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ...which, in turn, is a corner case that the test should cover. |
||
} | ||
|
||
int numBytes = 0; | ||
for (byte[] arr : operands) { | ||
numBytes += arr.length + 1; | ||
} | ||
numBytes--; | ||
|
||
byte[] result = new byte[numBytes]; | ||
|
||
System.arraycopy(operands.get(0), 0, result, 0, operands.get(0).length); | ||
|
||
for (int i = 1, arrIndex = operands.get(0).length; i < operands.size(); i++) { | ||
result[arrIndex] = DELIMITER; | ||
arrIndex += 1; | ||
System.arraycopy(operands.get(i), 0, result, arrIndex, operands.get(i).length); | ||
arrIndex += operands.get(i).length; | ||
} | ||
|
||
return result; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.contrib.streaming.state.benchmark; | ||
|
||
import org.apache.flink.contrib.streaming.state.PredefinedOptions; | ||
import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; | ||
import org.apache.flink.contrib.streaming.state.util.MergeUtils; | ||
import org.apache.flink.testutils.junit.RetryOnFailure; | ||
import org.apache.flink.testutils.junit.RetryRule; | ||
import org.apache.flink.util.TestLogger; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.rules.TemporaryFolder; | ||
import org.rocksdb.CompactionStyle; | ||
import org.rocksdb.NativeLibraryLoader; | ||
import org.rocksdb.Options; | ||
import org.rocksdb.RocksDB; | ||
import org.rocksdb.WriteOptions; | ||
|
||
import java.io.File; | ||
import java.nio.charset.StandardCharsets; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
|
||
/** | ||
* Test that validates that the performance of APIs of RocksDB's ListState is as expected. | ||
* | ||
* Benchmarking | ||
* Computer: MacbookPro (Mid 2015), Flash Storage, Processor 2.5GHz Intel Core i7, Memory 16GB 1600MHz DDR3 | ||
* | ||
* Number of values added | time for add() | time for update() | perf improvement of update() over add() | ||
* 500 978703 ns 55397 ns 17.66x | ||
* 1000 3044179 ns 89474 ns 34.02x | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure why the formatting is a few spaces behind others on github. They are aligned on my computer There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, that's an improvement that would justify the more complex approach. Nice! |
||
* 5000 9247395 ns 305580 ns 30.26x | ||
* 10000 16416442 ns 605963 ns 27.09x | ||
* 50000 84311205 ns 5691288 ns 14.81x | ||
* 100000 195103310 ns 12914182 ns 15.11x | ||
* 500000 1223141510 ns 70595881 ns 17.33x | ||
* | ||
* In summary, update() API which pre-merges all values gives users 15-35x performance improvements. | ||
* For most frequent use cases where there are a few hundreds to a few thousands values per key, | ||
* users can get 30x - 35x performance improvement! | ||
* | ||
*/ | ||
public class RocksDBListStatePerformanceTest extends TestLogger { | ||
|
||
@Rule | ||
public final TemporaryFolder tmp = new TemporaryFolder(); | ||
|
||
@Rule | ||
public final RetryRule retry = new RetryRule(); | ||
|
||
@Test(timeout = 2000) | ||
@RetryOnFailure(times = 3) | ||
public void testRocksDbListStateAPIs() throws Exception { | ||
final File rocksDir = tmp.newFolder(); | ||
|
||
// ensure the RocksDB library is loaded to a distinct location each retry | ||
NativeLibraryLoader.getInstance().loadLibrary(rocksDir.getAbsolutePath()); | ||
|
||
final String key1 = "key1"; | ||
final String key2 = "key2"; | ||
final String value = "abcdefghijklmnopqrstuvwxyz0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ7890654321"; | ||
|
||
final byte[] keyBytes1 = key1.getBytes(StandardCharsets.UTF_8); | ||
final byte[] keyBytes2 = key2.getBytes(StandardCharsets.UTF_8); | ||
final byte[] valueBytes = value.getBytes(StandardCharsets.UTF_8); | ||
|
||
// The number of values added to ListState. Can be changed for benchmarking | ||
final int num = 1000; | ||
|
||
try ( | ||
final Options options = new Options() | ||
.setCompactionStyle(CompactionStyle.LEVEL) | ||
.setLevelCompactionDynamicLevelBytes(true) | ||
.setIncreaseParallelism(4) | ||
.setUseFsync(false) | ||
.setMaxOpenFiles(-1) | ||
.setCreateIfMissing(true) | ||
.setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME); | ||
|
||
final WriteOptions writeOptions = new WriteOptions() | ||
.setSync(false) | ||
.setDisableWAL(true); | ||
|
||
final RocksDB rocksDB = RocksDB.open(options, rocksDir.getAbsolutePath())) { | ||
|
||
// ----- add() API ----- | ||
log.info("begin add"); | ||
System.out.println("begin add"); | ||
|
||
final long beginInsert1 = System.nanoTime(); | ||
for (int i = 0; i < num; i++) { | ||
rocksDB.merge(writeOptions, keyBytes1, valueBytes); | ||
} | ||
final long endInsert1 = System.nanoTime(); | ||
|
||
log.info("end add - duration: {} ns", (endInsert1 - beginInsert1)); | ||
|
||
// ----- update() API ----- | ||
|
||
List<byte[]> list = new ArrayList<>(num); | ||
for (int i = 0; i < num; i++) { | ||
list.add(valueBytes); | ||
} | ||
byte[] premerged = MergeUtils.merge(list); | ||
|
||
log.info("begin update"); | ||
|
||
final long beginInsert2 = System.nanoTime(); | ||
rocksDB.merge(writeOptions, keyBytes2, premerged); | ||
final long endInsert2 = System.nanoTime(); | ||
|
||
log.info("end update - duration: {} ns", (endInsert2 - beginInsert2)); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.contrib.streaming.state.util; | ||
|
||
import org.junit.Test; | ||
|
||
import java.util.Arrays; | ||
import java.util.List; | ||
|
||
import static org.junit.Assert.assertTrue; | ||
|
||
/** | ||
* Tests for MergeUtils. | ||
*/ | ||
public class MergeUtilsTest { | ||
@Test | ||
public void testMerge() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test should cover more corner cases, like empty list, singleton list, ... |
||
List<byte[]> list = Arrays.asList( | ||
new byte[4], | ||
new byte[1], | ||
new byte[2]); | ||
|
||
byte[] expected = new byte[9]; | ||
expected[4] = MergeUtils.DELIMITER; | ||
expected[6] = MergeUtils.DELIMITER; | ||
|
||
assertTrue(Arrays.equals(expected, MergeUtils.merge(list))); | ||
|
||
// Empty list | ||
list = Arrays.asList(); | ||
|
||
assertTrue(Arrays.equals(null, MergeUtils.merge(list))); | ||
|
||
// Singleton list | ||
list = Arrays.asList(new byte[1]); | ||
|
||
assertTrue(Arrays.equals(new byte[1], MergeUtils.merge(list))); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it expected behaviour that adding an empty list will not result in an entry? Maybe that is correct, but we should double check.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe it's the behavior in rocksdb. Good to have a second eye on it https://github.com/facebook/rocksdb/blob/master/utilities/merge_operators/string_append/stringappend2.cc since I never wrote c++ before
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What I meant, is not a RocksDB question, but a question if the list state should return
null
or empty iterator if after the user stored an empty list under a key. That might make a difference and right now, I the caller updates with empty list, the get is identical to a non-existing mapping (null
). Returning an empty list would require to make an entry in rocksdb.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
in that case, for all state backends, it returns
null
. I will add a unit test to verify and enforce it.