You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Arseniy Tashoyan (Jira)" <ji...@apache.org> on 2022/07/04 14:26:00 UTC

[jira] [Created] (FLINK-28383) HDFS source: Trying to access closed classloader

Arseniy Tashoyan created FLINK-28383:
----------------------------------------

             Summary: HDFS source: Trying to access closed classloader
                 Key: FLINK-28383
                 URL: https://issues.apache.org/jira/browse/FLINK-28383
             Project: Flink
          Issue Type: Bug
          Components: Connectors / FileSystem
    Affects Versions: 1.15.0
         Environment: Flink 1.15.0
            Reporter: Arseniy Tashoyan


This problem occurs when reading an Avro file from HDFS. Flink is not able to restart a failed task because of the exception mentioned below. As a result, the entire application fails.

{code:none}
Caused by: java.lang.IllegalStateException: Trying to access closed classloader. Please check if you store classloaders directly or indirectly in static fields. If the stacktrace suggests that the leak occurs in a third party library and cannot be fixed immediately, you can disable this check with the configuration 'classloader.check-leaked-classloader'.
	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164) ~[flink-dist-1.15.0.jar:1.15.0]
	at org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.loadClass(FlinkUserCodeClassLoaders.java:172) ~[flink-dist-1.15.0.jar:1.15.0]
	at java.lang.Class.forName0(Native Method) ~[?:1.8.0_252]
	at java.lang.Class.forName(Class.java:348) ~[?:1.8.0_252]
	at org.apache.hadoop.conf.Configuration.getClassByNameOrNull(Configuration.java:2247) ~[hadoop-common-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2212) ~[hadoop-common-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.crypto.CryptoCodec.getCodecClasses(CryptoCodec.java:125) ~[hadoop-common-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.crypto.CryptoCodec.getInstance(CryptoCodec.java:60) ~[hadoop-common-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.createStreamPair(DataTransferSaslUtil.java:339) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:511) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getEncryptedStreams(SaslDataTransferClient.java:304) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:245) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:215) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.net.TcpPeerServer.peerFromSocketAndKey(TcpPeerServer.java:93) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3568) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:779) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:696) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:359) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:669) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:888) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:945) ~[hadoop-hdfs-2.7.3.2.6.5.0-292.jar:?]
	at java.io.DataInputStream.read(DataInputStream.java:149) ~[?:1.8.0_252]
	at org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream.read(HadoopDataInputStream.java:96) ~[flink-dist-1.15.0.jar:1.15.0]
	at org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper.read(FSDataInputStreamWrapper.java:50) ~[flink-avro-1.15.0.jar:1.15.0]
	at org.apache.avro.file.DataFileReader.openReader(DataFileReader.java:65) ~[avro-1.10.2.jar:1.10.2]
	at org.apache.flink.formats.avro.AvroInputFormat.initReader(AvroInputFormat.java:135) ~[flink-avro-1.15.0.jar:1.15.0]
	at org.apache.flink.formats.avro.AvroInputFormat.open(AvroInputFormat.java:109) ~[flink-avro-1.15.0.jar:1.15.0]
{code}

The Hadoop class org.apache.hadoop.conf.Configuration has a member classLoader:

{code:java}
private ClassLoader classLoader;
  {
    classLoader = Thread.currentThread().getContextClassLoader();
    if (classLoader == null) {
      classLoader = Configuration.class.getClassLoader();
    }
  }
{code}

It seems that the threads are reused despite the classloader is already closed.





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