[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2024-05-17 Thread Thomas Weise (Jira)


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

Thomas Weise commented on FLINK-16686:
--

Flink 1.17:
{code:java}
Exception in thread "Thread-14" java.lang.IllegalArgumentException: classLoader 
cannot be null.
at com.esotericsoftware.kryo.Kryo.setClassLoader(Kryo.java:975)
at 
org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.checkKryoInitialized(KryoSerializer.java:550)
at 
org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.deserialize(KryoSerializer.java:391)
at 
org.apache.flink.api.common.typeutils.CompositeSerializer.deserialize(CompositeSerializer.java:156)
at 
org.apache.flink.contrib.streaming.state.ttl.RocksDbTtlCompactFiltersManager$ListElementFilter.nextElementLastAccessTimestamp(RocksDbTtlCompactFiltersManager.java:205)
at 
org.apache.flink.contrib.streaming.state.ttl.RocksDbTtlCompactFiltersManager$ListElementFilter.nextUnexpiredOffset(RocksDbTtlCompactFiltersManager.java:191)
 {code}

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3, 1.17.0
>Reporter: Andrey Zagrebin
>Priority: Not a Priority
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2024-02-16 Thread Emre Kartoglu (Jira)


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

Emre Kartoglu commented on FLINK-16686:
---

I saw this happen in a Flink 1.15 app today, I believe we should add 1.15 to 
"Affects Versions" list.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Not a Priority
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2023-08-31 Thread Alexis Sarda-Espinosa (Jira)


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

Alexis Sarda-Espinosa commented on FLINK-16686:
---

cc [~yunta]

I wanted to check if this problem occurs with Avro, so I did a similar 
experiment as the one I did before with Kryo. I only considered 
{{GenericRecord}} serialization for now, but I can confirm this problem also 
occurs in that case.

I basically implemented a custom serializer, similar to what's described [in 
this 
example|https://medium.com/wbaa/evolve-your-data-model-in-flinks-state-using-avro-f26982afa399],
 and after running a dummy job for 2 hours I got this exception:

{noformat}
AvroEmbeddedRocksDBTest > avroSerialization() STANDARD_ERROR
java.lang.NullPointerException
at org.apache.avro.Schema.applyAliases(Schema.java:1872)
at 
org.apache.avro.generic.GenericDatumReader.getResolver(GenericDatumReader.java:131)
at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:152)
at 
com.avro.flink.GenericRecordTypeSerializer.deserialize(GenericRecordTypeSerializer.kt:56)
at 
com.avro.flink.GenericRecordTypeSerializer.deserialize(GenericRecordTypeSerializer.kt:16)
at 
org.apache.flink.api.common.typeutils.CompositeSerializer.deserialize(CompositeSerializer.java:156)
at 
org.apache.flink.contrib.streaming.state.ttl.RocksDbTtlCompactFiltersManager$ListElementFilter.nextElementLastAccessTimestamp(RocksDbTtlCompactFiltersManager.java:205)
at 
org.apache.flink.contrib.streaming.state.ttl.RocksDbTtlCompactFiltersManager$ListElementFilter.nextUnexpiredOffset(RocksDbTtlCompactFiltersManager.java:191)
{noformat}

Note that this stacktrace does _not_ mention a classloader. Some more details 
from the dump in case they are interesting:

{noformat}
Current thread (0x7f7ae80b0800):  JavaThread "Thread-9949" [_thread_in_vm, 
id=9258, stack(0x7f7a59a01000,0x7f7a5a20)]

Stack: [0x7f7a59a01000,0x7f7a5a20],  sp=0x7f7a5a1fcdc0,  free 
space=8175k
Native frames: (J=compiled Java code, A=aot compiled Java code, j=interpreted, 
Vv=VM code, C=native code)
V  [libjvm.so+0x7b8f75]
V  [libjvm.so+0x980aab]
C  [librocksdbjni-linux64.so+0x222ce1]  
JavaListElementFilter::NextUnexpiredOffset(rocksdb::Slice const&, long, long) 
const+0x121
C  [librocksdbjni-linux64.so+0x648941]  
rocksdb::flink::FlinkCompactionFilter::ListDecide(rocksdb::Slice const&, 
std::string*) const+0x81
C  [librocksdbjni-linux64.so+0x648de7]  
rocksdb::flink::FlinkCompactionFilter::FilterV2(int, rocksdb::Slice const&, 
rocksdb::CompactionFilter::ValueType, rocksdb::Slice const&, std::string*, 
std::string*) const+0xc7
C  [librocksdbjni-linux64.so+0x3cce72]  
rocksdb::MergeHelper::FilterMerge(rocksdb::Slice const&, rocksdb::Slice 
const&)+0x172
C  [librocksdbjni-linux64.so+0x3cdf06]  
rocksdb::MergeHelper::MergeUntil(rocksdb::InternalIteratorBase*,
 rocksdb::CompactionRangeDelAggregator*, unsigned long, bool, bool)+0xcc6
C  [librocksdbjni-linux64.so+0x2d2a81]  
rocksdb::CompactionIterator::NextFromInput()+0x9a1
C  [librocksdbjni-linux64.so+0x2d4911]  
rocksdb::CompactionIterator::SeekToFirst()+0x11
C  [librocksdbjni-linux64.so+0x2dd2be]  
rocksdb::CompactionJob::ProcessKeyValueCompaction(rocksdb::CompactionJob::SubcompactionState*)+0x55e
C  [librocksdbjni-linux64.so+0x2de668]  rocksdb::CompactionJob::Run()+0x278
C  [librocksdbjni-linux64.so+0x33cec4]  
rocksdb::DBImpl::BackgroundCompaction(bool*, rocksdb::JobContext*, 
rocksdb::LogBuffer*, rocksdb::DBImpl::PrepickedCompaction*, 
rocksdb::Env::Priority)+0x1084
C  [librocksdbjni-linux64.so+0x3407e7]  
rocksdb::DBImpl::BackgroundCallCompaction(rocksdb::DBImpl::PrepickedCompaction*,
 rocksdb::Env::Priority)+0xd7
C  [librocksdbjni-linux64.so+0x340dda]  
rocksdb::DBImpl::BGWorkCompaction(void*)+0x3a
C  [librocksdbjni-linux64.so+0x5e5744]  
rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long)+0x254
C  [librocksdbjni-linux64.so+0x5e58ed]  
rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper(void*)+0x5d
{noformat}

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Not a Priority
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native 

[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-23 Thread Alexis Sarda-Espinosa (Jira)


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

Alexis Sarda-Espinosa commented on FLINK-16686:
---

It seems I spoke to soon, something definitely changed but I still get an 
exception, it's just that it's thrown after approximately 1 hour instead of 30 
seconds, not sure why.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-23 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-16686:
--

[~asardaes] The RocksDB version is different between Flink-1.13 and Flink-1.15, 
I am not sure whether this could impact the problem.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-23 Thread Alexis Sarda-Espinosa (Jira)


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

Alexis Sarda-Espinosa commented on FLINK-16686:
---

The test I did was using Flink 1.13 with a custom job and custom RocksDB 
configuration. With it, I could reliably get an exception after roughly 30 
seconds if I didn't use a custom serializer. After repeating the test with 
Flink 1.15, I no longer get the exception, so maybe something changed and 
TTL-compaction can work with Kryo now?

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-16 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-16686:
--

[~asardaes] To be honest, I am not sure whether a customized serializer could 
really walk around this problem. And this problem might be related with the 
implementation of kryo, which needs help from some kryo experts.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-16 Thread Alexis Sarda-Espinosa (Jira)


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

Alexis Sarda-Espinosa commented on FLINK-16686:
---

[~yunta] I did some limited testing and it seems a custom serializer works fine 
even if I provide it with my fat jar, but then I don't understand why it fails 
with Kryo, if it can use the custom serializer from the user classloader, can't 
it use Kryo as well?

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-14 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-16686:
--

[~asardaes] If I remember correctly, the problem would disappear if user 
switched to a customized serializer instead of the default kryo serializer. 
Maybe you can also try to reproduce this problem via kryo serializer and then 
switch the serializer to customized serializer.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2022-06-10 Thread Alexis Sarda-Espinosa (Jira)


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

Alexis Sarda-Espinosa commented on FLINK-16686:
---

The reported problems were falling back on Kryo, but since this ticket mentions 
"user class loader", would implementing a custom {{TypeInformation<>}} and 
{{TypeSerializer<>}} fail as well? Or could that work if those implementations 
are added to Flink's lib folder and thus loaded by the parent class loader?

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major, auto-deprioritized-minor
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



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


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-05-21 Thread Konstantin Knauf (Jira)


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

Konstantin Knauf commented on FLINK-16686:
--

Failing early sounds like a good option to me. [~sjwiesman] do you want to 
create a separate ticket for that?

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-05-20 Thread Till Rohrmann (Jira)


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

Till Rohrmann commented on FLINK-16686:
---

This could be an option to make the problem more visible to the user and 
helping him not to shoot himself in the foot until we have fixed the problem 
properly.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-05-20 Thread Seth Wiesman (Jira)


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

Seth Wiesman commented on FLINK-16686:
--

I can report users running into this again. I realize this is a tough problem 
to solve. In the interim, what about eagerly banning the use of Kryo serializer 
for List and Map state ttl's? As it already does not work, I don't believe this 
would be a breaking change but would prevent our users from corrupting their 
snapshots cc [~knaufk] [~trohrmann] [~liyu]

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0, 1.11.3, 1.13.0, 1.12.3
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-04-30 Thread David Anderson (Jira)


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

David Anderson commented on FLINK-16686:


Reported again in https://stackoverflow.com/q/67321286/2000823.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Minor
>  Labels: auto-deprioritized-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-04-29 Thread Flink Jira Bot (Jira)


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

Flink Jira Bot commented on FLINK-16686:


This issue was labeled "stale-major" 7 ago and has not received any updates so 
it is being deprioritized. If this ticket is actually Major, please raise the 
priority and ask a committer to assign you the issue or revive the public 
discussion.


> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Major
>  Labels: stale-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-04-22 Thread Flink Jira Bot (Jira)


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

Flink Jira Bot commented on FLINK-16686:


This major issue is unassigned and itself and all of its Sub-Tasks have not 
been updated for 30 days. So, it has been labeled "stale-major". If this ticket 
is indeed "major", please either assign yourself or give an update. Afterwards, 
please remove the label. In 7 days the issue will be deprioritized.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Major
>  Labels: stale-major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2021-02-22 Thread Dong Lin (Jira)


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

Dong Lin commented on FLINK-16686:
--

Discussed with [~yunta] about this bug. This bug is hard to fix partly because 
a proper solution requires changes to RocksDB and Kryo. It will take more time 
to agree on a coordinated solution across Flink, RocksDB and Kryo. Given that 
we have a workaround solution and the impact of this bug seems low (based on 
the number of comments in this thread), we have not initiated serious 
discussion in the Flink community to tackle this bug.

The workaround solution is what Andrey described above, i.e. write a custom 
serializer. 

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2020-05-07 Thread Andrey Zagrebin (Jira)


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

Andrey Zagrebin commented on FLINK-16686:
-

Unfortunately, I cannot suggest more than it is written in the original 
stackoverflow discussion:

As a workaround, I would suggest to use a list state with simple elements which 
type has a fixed byte length when serialized. That would be primitive types and 
POJOs, maybe implement a custom fixed length serializer if possible. Then the 
non-fixed length (Kryo or Chill) serializer does not have to be called in the 
TTL cleanup and the TTL cleanup would not require the user class loader.

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-16686) [State TTL] Make user class loader available in native RocksDB compaction thread

2020-05-06 Thread Roey Shem Tov (Jira)


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

Roey Shem Tov commented on FLINK-16686:
---

Just update it is happens on flink 1.10 too (Kubernetes mode).
Same exception as mentioned here 

{{}}
{code:java}
Falling back to default Kryo serializer because Chill serializer couldn't be 
found.
{code}
 

Any suggestion of how to solve it?

{{}}

> [State TTL] Make user class loader available in native RocksDB compaction 
> thread
> 
>
> Key: FLINK-16686
> URL: https://issues.apache.org/jira/browse/FLINK-16686
> Project: Flink
>  Issue Type: Bug
>  Components: Runtime / State Backends
>Affects Versions: 1.8.0
>Reporter: Andrey Zagrebin
>Priority: Major
>
> The issue is initially reported 
> [here|https://stackoverflow.com/questions/60745711/flink-kryo-serializer-because-chill-serializer-couldnt-be-found].
> The problem is that the java code of Flink compaction filter is called from 
> RocksDB native C++ code. It is called in the context of the native compaction 
> thread. RocksDB has utilities to create java Thread context for the Flink 
> java callback. Presumably, the Java thread context class loader is not set at 
> all and if it is queried then it produces NullPointerException.
> The provided report enabled a list state with TTL. The compaction filter has 
> to deserialise elements to check expiration. The deserialiser relies on Kryo 
> which queries the thread context class loader which is expected to be the 
> user class loader of the task but turns out to be null.
> We should investigate how to pass the user class loader to the compaction 
> thread of the list state with TTL.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)