You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by John Smith <ja...@gmail.com> on 2019/05/15 14:57:16 UTC

Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Hi,

I'm using vertx.io as an async JDBC client for a RichAsyncFunction it works
fine but when I stop the job I get...

java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
io/vertx/core/impl/VertxImpl$SharedWorkerPool

Is there a way to avoid/fix this?

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by John Smith <ja...@gmail.com>.
Thanks! This should do the trick...

@Override
public void close() throws Exception {
   CompletableFuture<Void> vertxClosed = new CompletableFuture<>();

   if(jdbc != null)
      jdbc.close();

   if(vertx != null)
      vertx.close(close -> {
         if(close.failed())
            logger.error("Vertx did not close properly!", close.cause());

         vertxClosed.complete(null);
      });

   if(ignite != null)
      ignite.close();

   // Give a chance to the async libs to close().
   vertxClosed.get(3000, TimeUnit.MILLISECONDS);
}


On Thu, 16 May 2019 at 15:25, Ken Krugler <kk...@transpac.com>
wrote:

>
> On May 16, 2019, at 9:38 AM, John Smith <ja...@gmail.com> wrote:
>
> Hi, so Thread.Sleep(1000) seems to do the trick. Now is this a good thing
> or bad thing?
>
>
> An arbitrary sleep duration has the potential to create random failures if
> the close takes longer than expected.
>
> For async close() calls, often you handle this by querying (in a loop,
> with sleep() and a max duration) the client until its status changes.
>
> — Ken
>
>
>
> On Thu, 16 May 2019 at 11:46, John Smith <ja...@gmail.com> wrote:
>
>> Yes when I mean cancel the JOB, it's when you go inside the UI and hit
>> the cancel button at the top right corner.
>>
>> The close is very simple...
>>
>> @Override
>> public void close() throws Exception {
>>    if(jdbc != null)
>>       jdbc.close();
>>
>>    if(vertx != null)
>>       vertx.close();
>>
>>    if(ignite != null)
>>       ignite.close();
>> }
>>
>> I was thinking that jdbc.close() and vertx.close() are actually async
>> underneath. The vertx APIs offers 2 methods for close, one which has async
>> interface and one that does not. But essentially they are both async.
>>
>> So I'm thinking maybe I can introduce a small Thread.Sleep(1000) after
>> the last close to give a chance for vertx.close().
>>
>>
>>
>>
>> On Thu, 16 May 2019 at 04:14, Andrey Zagrebin <an...@ververica.com>
>> wrote:
>>
>>> Could you share the source code of your RichAsyncFunction?
>>> Looks like netty threads of vertx are still being shutdown after user
>>> code class loader has been shutdown.
>>> It probably means that RichAsyncFunction was not closed properly or not
>>> all resources have been fully freed there (logging your
>>> RichAsyncFunction.close could help).
>>> Do you mean cancelation by stopping the job?
>>>
>>> On Wed, May 15, 2019 at 10:02 PM John Smith <ja...@gmail.com>
>>> wrote:
>>>
>>>> So these are the two exceptions I see in the logs...
>>>>
>>>> Exception in thread "vert.x-worker-thread-0" Exception in thread
>>>> "vert.x-internal-blocking-0" java.lang.NoClassDefFoundError:
>>>> io/netty/util/concurrent/FastThreadLocal
>>>> at
>>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>> Caused by: java.lang.ClassNotFoundException:
>>>> io.netty.util.concurrent.FastThreadLocal
>>>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>>> at
>>>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>>> ... 2 more
>>>> java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
>>>> at
>>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>> May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
>>>> SEVERE: Unhandled exception
>>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>> at
>>>> io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
>>>> at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
>>>> at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
>>>> at
>>>> io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
>>>> at
>>>> io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
>>>> at
>>>> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
>>>> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
>>>> at
>>>> io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
>>>> at
>>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>> Caused by: java.lang.NoClassDefFoundError:
>>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>> ... 10 more
>>>> Caused by: java.lang.ClassNotFoundException:
>>>> io.vertx.core.impl.VertxImpl$SharedWorkerPool
>>>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>>> at
>>>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>>> ... 10 more
>>>>
>>>> On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <an...@ververica.com>
>>>> wrote:
>>>>
>>>>> Hi John,
>>>>>
>>>>> could you share the full stack trace or better logs?
>>>>> It looks like something is trying to be executed in vertx.io code
>>>>> after the local task has been stopped and the class loader for the user
>>>>> code has been unloaded. Maybe from some daemon thread pool.
>>>>>
>>>>> Best,
>>>>> Andrey
>>>>>
>>>>>
>>>>> On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> I'm using vertx.io as an async JDBC client for a RichAsyncFunction
>>>>>> it works fine but when I stop the job I get...
>>>>>>
>>>>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>>>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>>>>
>>>>>> Is there a way to avoid/fix this?
>>>>>>
>>>>>
> --------------------------
> Ken Krugler
> +1 530-210-6378
> http://www.scaleunlimited.com
> Custom big data solutions & training
> Flink, Solr, Hadoop, Cascading & Cassandra
>
>

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by Ken Krugler <kk...@transpac.com>.
> On May 16, 2019, at 9:38 AM, John Smith <ja...@gmail.com> wrote:
> 
> Hi, so Thread.Sleep(1000) seems to do the trick. Now is this a good thing or bad thing?

An arbitrary sleep duration has the potential to create random failures if the close takes longer than expected.

For async close() calls, often you handle this by querying (in a loop, with sleep() and a max duration) the client until its status changes.

— Ken



> On Thu, 16 May 2019 at 11:46, John Smith <java.dev.mtl@gmail.com <ma...@gmail.com>> wrote:
> Yes when I mean cancel the JOB, it's when you go inside the UI and hit the cancel button at the top right corner.
> 
> The close is very simple...
> @Override
> public void close() throws Exception {
>    if(jdbc != null)
>       jdbc.close();
> 
>    if(vertx != null)
>       vertx.close();
> 
>    if(ignite != null)
>       ignite.close();
> }
> I was thinking that jdbc.close() and vertx.close() are actually async underneath. The vertx APIs offers 2 methods for close, one which has async interface and one that does not. But essentially they are both async.
> 
> So I'm thinking maybe I can introduce a small Thread.Sleep(1000) after the last close to give a chance for vertx.close().
> 
> 
> 
> 
> On Thu, 16 May 2019 at 04:14, Andrey Zagrebin <andrey@ververica.com <ma...@ververica.com>> wrote:
> Could you share the source code of your RichAsyncFunction?
> Looks like netty threads of vertx are still being shutdown after user code class loader has been shutdown.
> It probably means that RichAsyncFunction was not closed properly or not all resources have been fully freed there (logging your RichAsyncFunction.close could help).
> Do you mean cancelation by stopping the job?
> 
> On Wed, May 15, 2019 at 10:02 PM John Smith <java.dev.mtl@gmail.com <ma...@gmail.com>> wrote:
> So these are the two exceptions I see in the logs...
> 
> Exception in thread "vert.x-worker-thread-0" Exception in thread "vert.x-internal-blocking-0" java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
> 	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.ClassNotFoundException: io.netty.util.concurrent.FastThreadLocal
> 	at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> 	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> 	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
> 	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> 	... 2 more
> java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
> 	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
> 	at java.lang.Thread.run(Thread.java:748)
> May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
> SEVERE: Unhandled exception
> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError: io/vertx/core/impl/VertxImpl$SharedWorkerPool
> 	at io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
> 	at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
> 	at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
> 	at io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
> 	at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
> 	at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
> 	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
> 	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> 	at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NoClassDefFoundError: io/vertx/core/impl/VertxImpl$SharedWorkerPool
> 	... 10 more
> Caused by: java.lang.ClassNotFoundException: io.vertx.core.impl.VertxImpl$SharedWorkerPool
> 	at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> 	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> 	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
> 	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> 	... 10 more
> 
> On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <andrey@ververica.com <ma...@ververica.com>> wrote:
> Hi John,
> 
> could you share the full stack trace or better logs?
> It looks like something is trying to be executed in vertx.io <http://vertx.io/> code after the local task has been stopped and the class loader for the user code has been unloaded. Maybe from some daemon thread pool. 
> 
> Best,
> Andrey
> 
> 
> On Wed, May 15, 2019 at 4:58 PM John Smith <java.dev.mtl@gmail.com <ma...@gmail.com>> wrote:
> Hi, 
> 
> I'm using vertx.io <http://vertx.io/> as an async JDBC client for a RichAsyncFunction it works fine but when I stop the job I get...
> 
> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError: io/vertx/core/impl/VertxImpl$SharedWorkerPool
> 
> Is there a way to avoid/fix this?

--------------------------
Ken Krugler
+1 530-210-6378
http://www.scaleunlimited.com
Custom big data solutions & training
Flink, Solr, Hadoop, Cascading & Cassandra


Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by John Smith <ja...@gmail.com>.
Hi, so Thread.Sleep(1000) seems to do the trick. Now is this a good thing
or bad thing?

On Thu, 16 May 2019 at 11:46, John Smith <ja...@gmail.com> wrote:

> Yes when I mean cancel the JOB, it's when you go inside the UI and hit the
> cancel button at the top right corner.
>
> The close is very simple...
>
> @Override
> public void close() throws Exception {
>    if(jdbc != null)
>       jdbc.close();
>
>    if(vertx != null)
>       vertx.close();
>
>    if(ignite != null)
>       ignite.close();
> }
>
> I was thinking that jdbc.close() and vertx.close() are actually async
> underneath. The vertx APIs offers 2 methods for close, one which has async
> interface and one that does not. But essentially they are both async.
>
> So I'm thinking maybe I can introduce a small Thread.Sleep(1000) after the
> last close to give a chance for vertx.close().
>
>
>
>
> On Thu, 16 May 2019 at 04:14, Andrey Zagrebin <an...@ververica.com>
> wrote:
>
>> Could you share the source code of your RichAsyncFunction?
>> Looks like netty threads of vertx are still being shutdown after user
>> code class loader has been shutdown.
>> It probably means that RichAsyncFunction was not closed properly or not
>> all resources have been fully freed there (logging your
>> RichAsyncFunction.close could help).
>> Do you mean cancelation by stopping the job?
>>
>> On Wed, May 15, 2019 at 10:02 PM John Smith <ja...@gmail.com>
>> wrote:
>>
>>> So these are the two exceptions I see in the logs...
>>>
>>> Exception in thread "vert.x-worker-thread-0" Exception in thread
>>> "vert.x-internal-blocking-0" java.lang.NoClassDefFoundError:
>>> io/netty/util/concurrent/FastThreadLocal
>>> at
>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>>> at java.lang.Thread.run(Thread.java:748)
>>> Caused by: java.lang.ClassNotFoundException:
>>> io.netty.util.concurrent.FastThreadLocal
>>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>> at
>>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>> ... 2 more
>>> java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
>>> at
>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>>> at java.lang.Thread.run(Thread.java:748)
>>> May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
>>> SEVERE: Unhandled exception
>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>> at
>>> io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
>>> at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
>>> at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
>>> at
>>> io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
>>> at
>>> io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
>>> at
>>> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
>>> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
>>> at
>>> io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
>>> at
>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>>> at java.lang.Thread.run(Thread.java:748)
>>> Caused by: java.lang.NoClassDefFoundError:
>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>> ... 10 more
>>> Caused by: java.lang.ClassNotFoundException:
>>> io.vertx.core.impl.VertxImpl$SharedWorkerPool
>>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>> at
>>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>> ... 10 more
>>>
>>> On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <an...@ververica.com>
>>> wrote:
>>>
>>>> Hi John,
>>>>
>>>> could you share the full stack trace or better logs?
>>>> It looks like something is trying to be executed in vertx.io code
>>>> after the local task has been stopped and the class loader for the user
>>>> code has been unloaded. Maybe from some daemon thread pool.
>>>>
>>>> Best,
>>>> Andrey
>>>>
>>>>
>>>> On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> I'm using vertx.io as an async JDBC client for a RichAsyncFunction it
>>>>> works fine but when I stop the job I get...
>>>>>
>>>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>>>
>>>>> Is there a way to avoid/fix this?
>>>>>
>>>>

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by John Smith <ja...@gmail.com>.
Yes when I mean cancel the JOB, it's when you go inside the UI and hit the
cancel button at the top right corner.

The close is very simple...

@Override
public void close() throws Exception {
   if(jdbc != null)
      jdbc.close();

   if(vertx != null)
      vertx.close();

   if(ignite != null)
      ignite.close();
}

I was thinking that jdbc.close() and vertx.close() are actually async
underneath. The vertx APIs offers 2 methods for close, one which has async
interface and one that does not. But essentially they are both async.

So I'm thinking maybe I can introduce a small Thread.Sleep(1000) after the
last close to give a chance for vertx.close().




On Thu, 16 May 2019 at 04:14, Andrey Zagrebin <an...@ververica.com> wrote:

> Could you share the source code of your RichAsyncFunction?
> Looks like netty threads of vertx are still being shutdown after user code
> class loader has been shutdown.
> It probably means that RichAsyncFunction was not closed properly or not
> all resources have been fully freed there (logging your
> RichAsyncFunction.close could help).
> Do you mean cancelation by stopping the job?
>
> On Wed, May 15, 2019 at 10:02 PM John Smith <ja...@gmail.com>
> wrote:
>
>> So these are the two exceptions I see in the logs...
>>
>> Exception in thread "vert.x-worker-thread-0" Exception in thread
>> "vert.x-internal-blocking-0" java.lang.NoClassDefFoundError:
>> io/netty/util/concurrent/FastThreadLocal
>> at
>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>> at java.lang.Thread.run(Thread.java:748)
>> Caused by: java.lang.ClassNotFoundException:
>> io.netty.util.concurrent.FastThreadLocal
>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>> at
>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>> ... 2 more
>> java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
>> at
>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
>> at java.lang.Thread.run(Thread.java:748)
>> May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
>> SEVERE: Unhandled exception
>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>> at
>> io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
>> at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
>> at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
>> at
>> io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
>> at
>> io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
>> at
>> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
>> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
>> at
>> io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
>> at
>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>> at java.lang.Thread.run(Thread.java:748)
>> Caused by: java.lang.NoClassDefFoundError:
>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>> ... 10 more
>> Caused by: java.lang.ClassNotFoundException:
>> io.vertx.core.impl.VertxImpl$SharedWorkerPool
>> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>> at
>> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
>> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>> ... 10 more
>>
>> On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <an...@ververica.com>
>> wrote:
>>
>>> Hi John,
>>>
>>> could you share the full stack trace or better logs?
>>> It looks like something is trying to be executed in vertx.io code after
>>> the local task has been stopped and the class loader for the user code has
>>> been unloaded. Maybe from some daemon thread pool.
>>>
>>> Best,
>>> Andrey
>>>
>>>
>>> On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com>
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> I'm using vertx.io as an async JDBC client for a RichAsyncFunction it
>>>> works fine but when I stop the job I get...
>>>>
>>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>>
>>>> Is there a way to avoid/fix this?
>>>>
>>>

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by Andrey Zagrebin <an...@ververica.com>.
Could you share the source code of your RichAsyncFunction?
Looks like netty threads of vertx are still being shutdown after user code
class loader has been shutdown.
It probably means that RichAsyncFunction was not closed properly or not all
resources have been fully freed there (logging your RichAsyncFunction.close
could help).
Do you mean cancelation by stopping the job?

On Wed, May 15, 2019 at 10:02 PM John Smith <ja...@gmail.com> wrote:

> So these are the two exceptions I see in the logs...
>
> Exception in thread "vert.x-worker-thread-0" Exception in thread
> "vert.x-internal-blocking-0" java.lang.NoClassDefFoundError:
> io/netty/util/concurrent/FastThreadLocal
> at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.ClassNotFoundException:
> io.netty.util.concurrent.FastThreadLocal
> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> at
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> ... 2 more
> java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
> at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
> at java.lang.Thread.run(Thread.java:748)
> May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
> SEVERE: Unhandled exception
> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
> io/vertx/core/impl/VertxImpl$SharedWorkerPool
> at
> io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
> at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
> at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
> at
> io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
> at
> io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
> at
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
> at
> io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
> at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NoClassDefFoundError:
> io/vertx/core/impl/VertxImpl$SharedWorkerPool
> ... 10 more
> Caused by: java.lang.ClassNotFoundException:
> io.vertx.core.impl.VertxImpl$SharedWorkerPool
> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> at
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> ... 10 more
>
> On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <an...@ververica.com>
> wrote:
>
>> Hi John,
>>
>> could you share the full stack trace or better logs?
>> It looks like something is trying to be executed in vertx.io code after
>> the local task has been stopped and the class loader for the user code has
>> been unloaded. Maybe from some daemon thread pool.
>>
>> Best,
>> Andrey
>>
>>
>> On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com>
>> wrote:
>>
>>> Hi,
>>>
>>> I'm using vertx.io as an async JDBC client for a RichAsyncFunction it
>>> works fine but when I stop the job I get...
>>>
>>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>>
>>> Is there a way to avoid/fix this?
>>>
>>

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by John Smith <ja...@gmail.com>.
So these are the two exceptions I see in the logs...

Exception in thread "vert.x-worker-thread-0" Exception in thread
"vert.x-internal-blocking-0" java.lang.NoClassDefFoundError:
io/netty/util/concurrent/FastThreadLocal
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassNotFoundException:
io.netty.util.concurrent.FastThreadLocal
at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at
org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
... 2 more
java.lang.NoClassDefFoundError: io/netty/util/concurrent/FastThreadLocal
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:32)
at java.lang.Thread.run(Thread.java:748)
May 15, 2019 10:42:07 AM io.vertx.core.impl.ContextImpl
SEVERE: Unhandled exception
java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
io/vertx/core/impl/VertxImpl$SharedWorkerPool
at
io.vertx.core.impl.VertxImpl.lambda$deleteCacheDirAndShutdown$25(VertxImpl.java:830)
at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:284)
at io.vertx.core.impl.ContextImpl.executeTask(ContextImpl.java:320)
at
io.vertx.core.impl.EventLoopContext.lambda$executeAsync$0(EventLoopContext.java:38)
at
io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
at
io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:462)
at
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NoClassDefFoundError:
io/vertx/core/impl/VertxImpl$SharedWorkerPool
... 10 more
Caused by: java.lang.ClassNotFoundException:
io.vertx.core.impl.VertxImpl$SharedWorkerPool
at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at
org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$ChildFirstClassLoader.loadClass(FlinkUserCodeClassLoaders.java:129)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
... 10 more

On Wed, 15 May 2019 at 12:00, Andrey Zagrebin <an...@ververica.com> wrote:

> Hi John,
>
> could you share the full stack trace or better logs?
> It looks like something is trying to be executed in vertx.io code after
> the local task has been stopped and the class loader for the user code has
> been unloaded. Maybe from some daemon thread pool.
>
> Best,
> Andrey
>
>
> On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com> wrote:
>
>> Hi,
>>
>> I'm using vertx.io as an async JDBC client for a RichAsyncFunction it
>> works fine but when I stop the job I get...
>>
>> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
>> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>>
>> Is there a way to avoid/fix this?
>>
>

Re: Getting java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError when stopping/canceling job.

Posted by Andrey Zagrebin <an...@ververica.com>.
Hi John,

could you share the full stack trace or better logs?
It looks like something is trying to be executed in vertx.io code after the
local task has been stopped and the class loader for the user code has been
unloaded. Maybe from some daemon thread pool.

Best,
Andrey


On Wed, May 15, 2019 at 4:58 PM John Smith <ja...@gmail.com> wrote:

> Hi,
>
> I'm using vertx.io as an async JDBC client for a RichAsyncFunction it
> works fine but when I stop the job I get...
>
> java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError:
> io/vertx/core/impl/VertxImpl$SharedWorkerPool
>
> Is there a way to avoid/fix this?
>