@Yun Tang

Our FRocksDB has the same java package names (org.rocksdb.). Adding
'org.rocksdb' to parent-first patterns ensures it will be loaded only once,
and not accidentally multiple times (as Child-first classloading does).

The RocksDB code here is a bit like Flink internal components, which we
always force parent-first (everything that starts with "org.apache.fink.").

To use RocksDB from the application jar, I think you would need to remove
the RocksDB state backend from the classpath (lib folder), or you get
exactly the error reported above.

I cannot think of a downside to add RocksDB to the parent first patterns.

On Thu, Aug 5, 2021 at 10:04 AM Yun Tang <myas...@live.com> wrote:

> Hi Stephan,
>
> Since we use our own FRocksDB instead of the original RocksDB as
> dependency, I am not sure whether this problem has relationship with this.
> From my knowledge, more customers would include Flink classes within the
> application jar package, and it might cause problems if the client has
> different flink version with servers.
>
>
> Best,
> Yun Tang
> ------------------------------
> *From:* Stephan Ewen <se...@apache.org>
> *Sent:* Wednesday, August 4, 2021 19:10
> *To:* Yun Tang <myas...@live.com>
> *Cc:* Sandeep khanzode <sand...@shiftright.ai>; user <
> user@flink.apache.org>
> *Subject:* Re: Bloom Filter - RocksDB - LinkageError Classloading
>
> @Yun Tang Does it make sense to add RocksDB to the "always parent-first
> options" to avoid these kind of errors when users package apps incorrectly?
> My feeling is that these packaging errors occur very frequently.
>
>
> On Wed, Aug 4, 2021 at 10:41 AM Yun Tang <myas...@live.com> wrote:
>
> Hi Sandeep,
>
> Did you include the RocksDB classes in the application jar package? You
> can unpark your jar package to check whether them existed.
> If so, since RocksDB classes are already included in the flink-dist
> package, you don't need to include them in your jar package (maybe you
> explicitly added the dependency of org.rocksdb:rocksdbjni in your pom).
>
> Best
> Yun Tang
> ------------------------------
> *From:* Sandeep khanzode <sand...@shiftright.ai>
> *Sent:* Wednesday, August 4, 2021 11:54
> *To:* user <user@flink.apache.org>
> *Subject:* Bloom Filter - RocksDB - LinkageError Classloading
>
> Hello,
>
> I tried to add the bloom filter functionality as mentioned here:
>
> https://ci.apache.org/projects/flink/flink-docs-release-1.11/api/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactory.html
>
>  rocksDbBackend.setRocksDBOptions(new RocksDBOptionsFactory() {
>
>       public DBOptions createDBOptions(DBOptions currentOptions, 
> Collection<AutoCloseable> handlesToClose) {
>               return currentOptions.setMaxOpenFiles(1024);
>       }
>
>       public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
> currentOptions, Collection<AutoCloseable> handlesToClose) {
>               BloomFilter bloomFilter = new BloomFilter();
>                       handlesToClose.add(bloomFilter);
>
>                       return currentOptions
>                                       .setTableFormatConfig(
>                                                       new 
> BlockBasedTableConfig().setFilter(bloomFilter));
>       }
>  });
>
>
> This is in the main class where we setup in the StreamExecutionEnvironment
> …
>
> I get ClassLoading errors due to that ...
>
> Caused by: java.lang.LinkageError: loader constraint violation: loader 
> org.apache.flink.util.ChildFirstClassLoader @1169afe1 wants to load class 
> org.rocksdb.ColumnFamilyOptions. A different class with the same name was 
> previously loaded by 'app'. (org.rocksdb.ColumnFamilyOptions is in unnamed 
> module of loader 'app')
>
>
>
> What is documented is to change the order to parent-first in the
> flink-conf.yaml … but then I get different issues for the basic/core Spring
> Framework classes not being serializable …
>
> Any help will be appreciated.
>
> Thanks,
> Sandip
>
>

Reply via email to