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

Matthias Pohl commented on FLINK-29835:
---------------------------------------

Several stacktrace show up in the logs:

NoClassDefFoundError (not exclusively for 
{{{}SingletonContext$GenerationComparator{}}}!):
{code:java}
 2022-12-03 03:54:32,784 WARN  
org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClient [] - Client 
shutdown hook java.lang.ref.WeakReference failed.
 java.lang.NoClassDefFoundError: 
org/apache/pulsar/shade/org/jvnet/hk2/internal/SingletonContext$GenerationComparator
   at 
org.apache.pulsar.shade.org.jvnet.hk2.internal.SingletonContext.shutdown(SingletonContext.java:142)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.jvnet.hk2.internal.ServiceLocatorImpl.shutdown(ServiceLocatorImpl.java:920)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.inject.hk2.AbstractHk2InjectionManager.shutdown(AbstractHk2InjectionManager.java:183)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.inject.hk2.ImmediateHk2InjectionManager.shutdown(ImmediateHk2InjectionManager.java:30)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.client.ClientRuntime.close(ClientRuntime.java:371)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.client.ClientRuntime.onShutdown(ClientRuntime.java:353)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClient.release(JerseyClient.java:190)
 ~[?:?]
   at 
org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClient.close(JerseyClient.java:180)
 ~[?:?]
   at 
org.apache.pulsar.client.admin.internal.PulsarAdminImpl.close(PulsarAdminImpl.java:488)
 ~[?:?]
   at 
org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator.close(PulsarSourceEnumerator.java:164)
 ~[?:?]
   at org.apache.flink.util.IOUtils.closeAll(IOUtils.java:254) 
~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at 
org.apache.flink.runtime.source.coordinator.SourceCoordinator.close(SourceCoordinator.java:216)
 ~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at 
org.apache.flink.runtime.operators.coordination.ComponentClosingUtils.lambda$closeAsyncWithTimeout$0(ComponentClosingUtils.java:76)
 ~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at java.lang.Thread.run(Thread.java:750) [?:1.8.0_342]
 Caused by: java.lang.ClassNotFoundException: 
org.apache.pulsar.shade.org.jvnet.hk2.internal.SingletonContext$GenerationComparator
   at java.net.URLClassLoader.findClass(URLClassLoader.java:387) ~[?:1.8.0_342]
   at java.lang.ClassLoader.loadClass(ClassLoader.java:418) ~[?:1.8.0_342]
   at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClassWithoutExceptionHandling(FlinkUserCodeClassLoader.java:68)
 ~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at 
org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:74)
 ~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at 
org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:52)
 ~[flink-dist-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
   at java.lang.ClassLoader.loadClass(ClassLoader.java:351) ~[?:1.8.0_342]
   ... 14 more{code}
{{{}CoordinatorException$InvalidTxnStatusException{}}}:
{code:java}
2022-12-03T03:54:39,784+0000 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR 
org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore - 
TxnID : (0,3643) add acked subscription error with TxnStatus : COMMITTING
org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException$InvalidTxnStatusException:
 Expect Txn `(0,3643)` to be in OPEN status but it is in COMMITTING status
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.checkTxnStatus(TxnMetaImpl.java:96)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.addAckedPartitions(TxnMetaImpl.java:127)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.addAckedPartitions(TxnMetaImpl.java:37)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore.lambda$addAckedPartitionToTxn$7(MLTransactionMetadataStore.java:330)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714)
 ~[?:?]
   at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]
   at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) 
~[?:?]
   at 
org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl$3.addComplete(MLTransactionLogImpl.java:158)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.bookkeeper.mledger.impl.OpAddEntry.safeRun(OpAddEntry.java:232) 
~[org.apache.pulsar-managed-ledger-2.10.2.jar:2.10.2]
   at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) 
~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
~[?:?]
   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
~[?:?]
   at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
   at java.lang.Thread.run(Thread.java:829) ~[?:?]
2022-12-03T03:54:39,784+0000 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR 
org.apache.pulsar.broker.service.ServerCnx - Send response error for 
ADD_SUBSCRIPTION_TO_TXN request 2032153913178166290.
org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException$InvalidTxnStatusException:
 Expect Txn `(0,3643)` to be in OPEN status but it is in COMMITTING status
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.checkTxnStatus(TxnMetaImpl.java:96)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.addAckedPartitions(TxnMetaImpl.java:127)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.TxnMetaImpl.addAckedPartitions(TxnMetaImpl.java:37)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore.lambda$addAckedPartitionToTxn$7(MLTransactionMetadataStore.java:330)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714)
 ~[?:?]
   at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]
   at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) 
~[?:?]
   at 
org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl$3.addComplete(MLTransactionLogImpl.java:158)
 ~[org.apache.pulsar-pulsar-transaction-coordinator-2.10.2.jar:2.10.2]
   at 
org.apache.bookkeeper.mledger.impl.OpAddEntry.safeRun(OpAddEntry.java:232) 
~[org.apache.pulsar-managed-ledger-2.10.2.jar:2.10.2]
   at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) 
~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
~[?:?]
   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
~[?:?]
   at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
   at java.lang.Thread.run(Thread.java:829) ~[?:?] {code}

> NoClassDefFoundError in PulsarSourceUnorderedE2ECase
> ----------------------------------------------------
>
>                 Key: FLINK-29835
>                 URL: https://issues.apache.org/jira/browse/FLINK-29835
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Pulsar, Runtime / Coordination
>    Affects Versions: 1.16.0, 1.17.0
>            Reporter: Matthias Pohl
>            Priority: Critical
>              Labels: test-stability
>         Attachments: 
> PulsarSourceUnorderedE2ECase.testSavepoint.FileNotFoundException.log
>
>
> [This 
> build|https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=42680&view=logs&j=87489130-75dc-54e4-1f45-80c30aa367a3&t=73da6d75-f30d-5d5a-acbe-487a9dcff678&l=16001]
>  failed in {{PulsarSourceUnorderedE2ECase.testSavepoint}} due to some job 
> timeout.
> The logs reveal {{{}NoClassDefFoundErrors{}}}:
> {code:java}
> 2022-11-14 15:36:59,696 WARN  
> org.apache.pulsar.shade.org.asynchttpclient.DefaultAsyncHttpClient [] - 
> Unexpected error on ChannelManager close
> java.lang.NoClassDefFoundError: 
> org/apache/pulsar/shade/io/netty/util/concurrent/DefaultPromise$1
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:499)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:616)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.DefaultPromise.setFailure0(DefaultPromise.java:609)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:117)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.ensureThreadStarted(SingleThreadEventExecutor.java:970)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor.shutdownGracefully(SingleThreadEventExecutor.java:661)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.io.netty.util.concurrent.MultithreadEventExecutorGroup.shutdownGracefully(MultithreadEventExecutorGroup.java:163)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.org.asynchttpclient.netty.channel.ChannelManager.close(ChannelManager.java:307)
>  ~[?:?]
>      at 
> org.apache.pulsar.shade.org.asynchttpclient.DefaultAsyncHttpClient.close(DefaultAsyncHttpClient.java:120)
>  ~[?:?]
>      at 
> org.apache.pulsar.client.admin.internal.http.AsyncHttpConnector.close(AsyncHttpConnector.java:346)
>  ~[?:?]
>      at 
> org.apache.pulsar.client.admin.internal.PulsarAdminImpl.close(PulsarAdminImpl.java:490)
>  ~[?:?]
>      at 
> org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator.close(PulsarSourceEnumerator.java:172)
>  ~[?:?]
>      at org.apache.flink.util.IOUtils.closeAll(IOUtils.java:255) 
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at 
> org.apache.flink.runtime.source.coordinator.SourceCoordinator.close(SourceCoordinator.java:265)
>  ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at 
> org.apache.flink.runtime.operators.coordination.ComponentClosingUtils.lambda$closeAsyncWithTimeout$0(ComponentClosingUtils.java:76)
>  ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at java.lang.Thread.run(Thread.java:750) [?:1.8.0_342]
> Caused by: java.lang.ClassNotFoundException: 
> org.apache.pulsar.shade.io.netty.util.concurrent.DefaultPromise$1
>      at java.net.URLClassLoader.findClass(URLClassLoader.java:387) 
> ~[?:1.8.0_342]
>      at java.lang.ClassLoader.loadClass(ClassLoader.java:418) ~[?:1.8.0_342]
>      at 
> org.apache.flink.util.FlinkUserCodeClassLoader.loadClassWithoutExceptionHandling(FlinkUserCodeClassLoader.java:67)
>  ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at 
> org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:74)
>  ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at 
> org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:51)
>  ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
>      at java.lang.ClassLoader.loadClass(ClassLoader.java:351) ~[?:1.8.0_342]
>      ... 16 more {code}
> Outdated:
> -The issue seems to be related to the BlobServer failing to provide some 
> artifacts ({{{}java.io.FileNotFoundException{}}}) that consequently causes 
> classes not being found.-
>  



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

Reply via email to