[ 
https://issues.apache.org/jira/browse/FLINK-24932?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17557331#comment-17557331
 ] 

Sebastian Mattheis edited comment on FLINK-24932 at 6/22/22 9:16 AM:
---------------------------------------------------------------------

[~yunta] Sorry for getting back so late. (The benchmarks Jenkins cluster was 
down and we had some trouble to get it back running because we simply had no 
access to the machines.) I have run java-8 benchmarks and backend benchmarks 
(some time ago). There was some performance decrease in state backend 
benchmarks where run #364 uses the bumped frocksdb version which I compared to 
a previous time triggered run #363. There is a decrease of ~6% in mapContains, 
~8% decrease in mapEntries, MapKeys, mapValues, and mapIterator and ~10% 
decrease in mapIsEmpty.

Please have a look and let me know what you think about the decrease. In my 
opinion, the performance decrease results from RocksDB changes (nothing else 
changed) and it's rather the question if we can accept this decrease and if we 
have an alternative.

 
|Benchmark|Mode|Threads|Samples|Score usual (#363)|Score|Score diff|Score Error 
(99.9%)|Unit|Param: backendType|Param: rescaleType|
|org.apache.flink.state.benchmark.ListStateBenchmark.listAdd|thrpt|1|30|6232.433058|6138.795822|-1.50%|38.00193|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAdd|thrpt|1|30|615.591947|622.875462|1.18%|4.737319|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAddAll|thrpt|1|30|978.45937|1028.899247|5.16%|29.629799|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAddAll|thrpt|1|30|413.652566|422.068031|2.03%|4.998573|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAppend|thrpt|1|30|3174.071023|3169.486257|-0.14%|16.652019|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAppend|thrpt|1|30|585.182107|587.940191|0.47%|7.711719|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGet|thrpt|1|30|3160.552545|3153.799272|-0.21%|17.486431|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGet|thrpt|1|30|167.080554|163.842208|-1.94%|1.456977|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGetAndIterate|thrpt|1|30|3095.508926|3102.200739|0.22%|23.689638|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGetAndIterate|thrpt|1|30|166.552043|162.699961|-2.31%|1.226418|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listUpdate|thrpt|1|30|3188.055401|3154.588841|-1.05%|18.473072|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listUpdate|thrpt|1|30|611.669153|612.994932|0.22%|5.979561|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapAdd|thrpt|1|30|3124.951349|3111.251285|-0.44%|49.236153|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapAdd|thrpt|1|30|527.202883|523.35679|-0.73%|26.435825|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapContains|thrpt|1|30|2971.568697|2946.448963|-0.85%|41.386895|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapContains|thrpt|1|30|65.105509|61.431574|-5.64%|0.207701|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapEntries|thrpt|1|30|17061.283367|16956.38978|-0.61%|163.697678|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapEntries|thrpt|1|30|451.837024|416.406964|-7.84%|7.654164|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapGet|thrpt|1|30|2771.22|2776.42085|0.19%|33.795728|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapGet|thrpt|1|30|65.774909|63.58201|-3.33%|0.737758|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIsEmpty|thrpt|1|30|3757.18286|3711.537421|-1.21%|67.411138|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIsEmpty|thrpt|1|30|57.614486|51.839015|-10.02%|0.114671|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIterator|thrpt|1|30|16341.927292|16353.069615|0.07%|292.760044|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIterator|thrpt|1|30|449.876148|411.067795|-8.63%|6.771682|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapKeys|thrpt|1|30|17536.594826|17678.088225|0.81%|146.846679|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapKeys|thrpt|1|30|456.047584|419.510585|-8.01%|8.067556|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapPutAll|thrpt|1|30|1884.636418|1880.097908|-0.24%|17.423581|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapPutAll|thrpt|1|30|161.319731|160.441502|-0.54%|7.390766|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapRemove|thrpt|1|30|3556.601649|3573.590858|0.48%|73.845766|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapRemove|thrpt|1|30|525.845489|520.629298|-0.99%|29.099397|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapUpdate|thrpt|1|30|2729.192789|2729.263963|0.00%|65.316609|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapUpdate|thrpt|1|30|515.601085|522.163984|1.27%|25.669994|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapValues|thrpt|1|30|17726.605443|17589.599688|-0.77%|120.801263|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapValues|thrpt|1|30|448.887827|413.972114|-7.78%|6.827274|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueAdd|thrpt|1|30|3773.706924|3757.139133|-0.44%|117.294596|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueAdd|thrpt|1|30|524.628733|518.543701|-1.16%|29.094959|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueGet|thrpt|1|30|4067.594929|4076.65271|0.22%|79.193942|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueGet|thrpt|1|30|865.19527|826.982382|-4.42%|9.202046|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueUpdate|thrpt|1|30|4168.748094|4164.319832|-0.11%|110.268518|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueUpdate|thrpt|1|30|522.284488|528.90659|1.27%|28.11252|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.HashMapStateBackendRescalingBenchmarkExecutor.rescaleHeap|avgt|1|30|9487.369168|9514.35009|0.28%|288.563841|ms/op|
 |RESCALE_IN|
|org.apache.flink.state.benchmark.HashMapStateBackendRescalingBenchmarkExecutor.rescaleHeap|avgt|1|30|4747.983584|4759.447415|0.24%|212.877985|ms/op|
 |RESCALE_OUT|
|org.apache.flink.state.benchmark.RocksdbStateBackendRescalingBenchmarkExecutor.rescaleRocksDB|avgt|1|30|32115.616436|31421.539981|-2.16%|603.468807|ms/op|
 |RESCALE_IN|
|org.apache.flink.state.benchmark.RocksdbStateBackendRescalingBenchmarkExecutor.rescaleRocksDB|avgt|1|30|590.948639|594.15776|0.54%|10.76513|ms/op|
 |RESCALE_OUT|

 

 


was (Author: JIRAUSER284806):
[~yunta] Sorry for getting back so late. (The benchmarks Jenkins cluster was 
done and we had some trouble to get it back running because we simply had no 
access to the machines.) I have run java-8 benchmarks and backend benchmarks 
(some time ago). There was some performance decrease in state backend 
benchmarks where run #364 uses the bumped frocksdb version which I compared to 
a previous time triggered run #363. There is a decrease of ~6% in mapContains, 
~8% decrease in mapEntries, MapKeys, mapValues, and mapIterator and ~10% 
decrease in mapIsEmpty.



Please have a look and let me know what you think about the decrease. In my 
opinion, the performance decrease results from RocksDB changes (nothing else 
changed) and it's rather the question if we can accept this decrease and if we 
have an alternative.

 
         |Benchmark|Mode|Threads|Samples|Score usual (#363)|Score|Score 
diff|Score Error (99.9%)|Unit|Param: backendType|Param: rescaleType|
|org.apache.flink.state.benchmark.ListStateBenchmark.listAdd|thrpt|1|30|6232.433058|6138.795822|-1.50%|38.00193|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAdd|thrpt|1|30|615.591947|622.875462|1.18%|4.737319|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAddAll|thrpt|1|30|978.45937|1028.899247|5.16%|29.629799|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAddAll|thrpt|1|30|413.652566|422.068031|2.03%|4.998573|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAppend|thrpt|1|30|3174.071023|3169.486257|-0.14%|16.652019|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listAppend|thrpt|1|30|585.182107|587.940191|0.47%|7.711719|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGet|thrpt|1|30|3160.552545|3153.799272|-0.21%|17.486431|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGet|thrpt|1|30|167.080554|163.842208|-1.94%|1.456977|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGetAndIterate|thrpt|1|30|3095.508926|3102.200739|0.22%|23.689638|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listGetAndIterate|thrpt|1|30|166.552043|162.699961|-2.31%|1.226418|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listUpdate|thrpt|1|30|3188.055401|3154.588841|-1.05%|18.473072|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ListStateBenchmark.listUpdate|thrpt|1|30|611.669153|612.994932|0.22%|5.979561|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapAdd|thrpt|1|30|3124.951349|3111.251285|-0.44%|49.236153|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapAdd|thrpt|1|30|527.202883|523.35679|-0.73%|26.435825|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapContains|thrpt|1|30|2971.568697|2946.448963|-0.85%|41.386895|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapContains|thrpt|1|30|65.105509|61.431574|-5.64%|0.207701|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapEntries|thrpt|1|30|17061.283367|16956.38978|-0.61%|163.697678|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapEntries|thrpt|1|30|451.837024|416.406964|-7.84%|7.654164|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapGet|thrpt|1|30|2771.22|2776.42085|0.19%|33.795728|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapGet|thrpt|1|30|65.774909|63.58201|-3.33%|0.737758|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIsEmpty|thrpt|1|30|3757.18286|3711.537421|-1.21%|67.411138|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIsEmpty|thrpt|1|30|57.614486|51.839015|-10.02%|0.114671|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIterator|thrpt|1|30|16341.927292|16353.069615|0.07%|292.760044|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapIterator|thrpt|1|30|449.876148|411.067795|-8.63%|6.771682|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapKeys|thrpt|1|30|17536.594826|17678.088225|0.81%|146.846679|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapKeys|thrpt|1|30|456.047584|419.510585|-8.01%|8.067556|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapPutAll|thrpt|1|30|1884.636418|1880.097908|-0.24%|17.423581|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapPutAll|thrpt|1|30|161.319731|160.441502|-0.54%|7.390766|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapRemove|thrpt|1|30|3556.601649|3573.590858|0.48%|73.845766|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapRemove|thrpt|1|30|525.845489|520.629298|-0.99%|29.099397|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapUpdate|thrpt|1|30|2729.192789|2729.263963|0.00%|65.316609|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapUpdate|thrpt|1|30|515.601085|522.163984|1.27%|25.669994|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapValues|thrpt|1|30|17726.605443|17589.599688|-0.77%|120.801263|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.MapStateBenchmark.mapValues|thrpt|1|30|448.887827|413.972114|-7.78%|6.827274|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueAdd|thrpt|1|30|3773.706924|3757.139133|-0.44%|117.294596|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueAdd|thrpt|1|30|524.628733|518.543701|-1.16%|29.094959|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueGet|thrpt|1|30|4067.594929|4076.65271|0.22%|79.193942|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueGet|thrpt|1|30|865.19527|826.982382|-4.42%|9.202046|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueUpdate|thrpt|1|30|4168.748094|4164.319832|-0.11%|110.268518|ops/ms|HEAP|
 |
|org.apache.flink.state.benchmark.ValueStateBenchmark.valueUpdate|thrpt|1|30|522.284488|528.90659|1.27%|28.11252|ops/ms|ROCKSDB|
 |
|org.apache.flink.state.benchmark.HashMapStateBackendRescalingBenchmarkExecutor.rescaleHeap|avgt|1|30|9487.369168|9514.35009|0.28%|288.563841|ms/op|
 |RESCALE_IN|
|org.apache.flink.state.benchmark.HashMapStateBackendRescalingBenchmarkExecutor.rescaleHeap|avgt|1|30|4747.983584|4759.447415|0.24%|212.877985|ms/op|
 |RESCALE_OUT|
|org.apache.flink.state.benchmark.RocksdbStateBackendRescalingBenchmarkExecutor.rescaleRocksDB|avgt|1|30|32115.616436|31421.539981|-2.16%|603.468807|ms/op|
 |RESCALE_IN|
|org.apache.flink.state.benchmark.RocksdbStateBackendRescalingBenchmarkExecutor.rescaleRocksDB|avgt|1|30|590.948639|594.15776|0.54%|10.76513|ms/op|
 |RESCALE_OUT|

 

 

> Frocksdb cannot run on Apple M1
> -------------------------------
>
>                 Key: FLINK-24932
>                 URL: https://issues.apache.org/jira/browse/FLINK-24932
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Runtime / State Backends
>            Reporter: Yun Tang
>            Assignee: Sebastian Mattheis
>            Priority: Minor
>
> After we bump up RocksDB version to 6.20.3, we support to run RocksDB on 
> linux arm cluster. However, according to the feedback from Robert, Apple M1 
> machines cannot run FRocksDB yet:
> {code:java}
> java.lang.Exception: Exception while creating StreamOperatorStateContext.
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:255)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:268)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:109)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:711)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:687)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:654)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958)
>  ~[flink-runtime-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927) 
> ~[flink-runtime-1.14.0.jar:1.14.0]
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) 
> ~[flink-runtime-1.14.0.jar:1.14.0]
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) 
> ~[flink-runtime-1.14.0.jar:1.14.0]
>       at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_312]
> Caused by: org.apache.flink.util.FlinkException: Could not restore keyed 
> state backend for StreamFlatMap_c21234bcbf1e8eb4c61f1927190efebd_(1/1) from 
> any of the 1 provided restore options.
>       at 
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:160)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:346)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:164)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       ... 11 more
> Caused by: java.io.IOException: Could not load the native RocksDB library
>       at 
> org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend.ensureRocksDBIsLoaded(EmbeddedRocksDBStateBackend.java:882)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend.createKeyedStateBackend(EmbeddedRocksDBStateBackend.java:402)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:345)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:87)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:329)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:168)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:346)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:164)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       ... 11 more
> Caused by: java.lang.UnsatisfiedLinkError: 
> /private/var/folders/js/yfk_y2450q7559kygttykwk00000gn/T/rocksdb-lib-5783c058ce68d31d371327abc9b51cac/librocksdbjni-osx.jnilib:
>  
> dlopen(/private/var/folders/js/yfk_y2450q7559kygttykwk00000gn/T/rocksdb-lib-5783c058ce68d31d371327abc9b51cac/librocksdbjni-osx.jnilib,
>  0x0001): tried: 
> '/private/var/folders/js/yfk_y2450q7559kygttykwk00000gn/T/rocksdb-lib-5783c058ce68d31d371327abc9b51cac/librocksdbjni-osx.jnilib'
>  (mach-o file, but is an incompatible architecture (have 'x86_64', need 
> 'arm64e')), '/usr/lib/librocksdbjni-osx.jnilib' (no such file)
>       at java.lang.ClassLoader$NativeLibrary.load(Native Method) 
> ~[?:1.8.0_312]
>       at java.lang.ClassLoader.loadLibrary0(ClassLoader.java:1950) 
> ~[?:1.8.0_312]
>       at java.lang.ClassLoader.loadLibrary(ClassLoader.java:1832) 
> ~[?:1.8.0_312]
>       at java.lang.Runtime.load0(Runtime.java:811) ~[?:1.8.0_312]
>       at java.lang.System.load(System.java:1088) ~[?:1.8.0_312]
>       at 
> org.rocksdb.NativeLibraryLoader.loadLibraryFromJar(NativeLibraryLoader.java:79)
>  ~[frocksdbjni-6.20.3-ververica-1.0.jar:?]
>       at 
> org.rocksdb.NativeLibraryLoader.loadLibrary(NativeLibraryLoader.java:57) 
> ~[frocksdbjni-6.20.3-ververica-1.0.jar:?]
>       at 
> org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend.ensureRocksDBIsLoaded(EmbeddedRocksDBStateBackend.java:856)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend.createKeyedStateBackend(EmbeddedRocksDBStateBackend.java:402)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:345)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:87)
>  ~[flink-statebackend-rocksdb_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:329)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:168)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:346)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       at 
> org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:164)
>  ~[flink-streaming-java_2.11-1.14.0.jar:1.14.0]
>       ... 11 more
> {code}
> This issue is tracked by RocksDB community: 
> [rocksdb/issues/7720|https://github.com/facebook/rocksdb/issues/7720]



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to