[ https://issues.apache.org/jira/browse/HDFS-17630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17884187#comment-17884187 ]
ASF GitHub Bot commented on HDFS-17630: --------------------------------------- cxzl25 commented on PR #7063: URL: https://github.com/apache/hadoop/pull/7063#issuecomment-2370670841 Spark uses FsUrlStreamHandlerFactory to support HDFS Jar, but in some scenarios PacketReceiver will be called nested, causing Spark to fail to start. cc @sunchao https://github.com/apache/spark/blob/982028ea7fc61d7aa84756aa46860ebb49bfe9d1/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L201 <details> <summary>PacketReceiver Exception</summary> ```java java.lang.Exception at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:166) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:112) at org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.readNextPacket(BlockReaderRemote.java:187) at org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.read(BlockReaderRemote.java:146) at org.apache.hadoop.hdfs.ByteArrayStrategy.readFromBlock(ReaderStrategy.java:118) at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:789) at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:855) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:919) at java.base/java.io.DataInputStream.read(DataInputStream.java:158) at java.base/java.io.InputStream.transferTo(InputStream.java:796) at java.base/java.nio.file.Files.copy(Files.java:3151) at java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:216) at java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:212) at java.base/java.security.AccessController.doPrivileged(AccessController.java:571) at java.base/sun.net.www.protocol.jar.URLJarFile.retrieve(URLJarFile.java:211) at java.base/sun.net.www.protocol.jar.URLJarFile.getJarFile(URLJarFile.java:71) at java.base/sun.net.www.protocol.jar.JarFileFactory.get(JarFileFactory.java:153) at java.base/sun.net.www.protocol.jar.JarURLConnection.connect(JarURLConnection.java:109) at java.base/sun.net.www.protocol.jar.JarURLConnection.getJarFile(JarURLConnection.java:70) at java.base/jdk.internal.loader.URLClassPath$JarLoader.getJarFile(URLClassPath.java:814) at java.base/jdk.internal.loader.URLClassPath$JarLoader$1.run(URLClassPath.java:774) at java.base/jdk.internal.loader.URLClassPath$JarLoader$1.run(URLClassPath.java:768) at java.base/java.security.AccessController.doPrivileged(AccessController.java:714) at java.base/jdk.internal.loader.URLClassPath$JarLoader.ensureOpen(URLClassPath.java:767) at java.base/jdk.internal.loader.URLClassPath$JarLoader.<init>(URLClassPath.java:734) at java.base/jdk.internal.loader.URLClassPath$3.run(URLClassPath.java:497) at java.base/jdk.internal.loader.URLClassPath$3.run(URLClassPath.java:479) at java.base/java.security.AccessController.doPrivileged(AccessController.java:714) at java.base/jdk.internal.loader.URLClassPath.getLoader(URLClassPath.java:478) at java.base/jdk.internal.loader.URLClassPath.getLoader(URLClassPath.java:446) at java.base/jdk.internal.loader.URLClassPath.findResource(URLClassPath.java:292) at java.base/java.net.URLClassLoader$2.run(URLClassLoader.java:629) at java.base/java.net.URLClassLoader$2.run(URLClassLoader.java:627) at java.base/java.security.AccessController.doPrivileged(AccessController.java:400) at java.base/java.net.URLClassLoader.findResource(URLClassLoader.java:626) at java.base/java.lang.ClassLoader.getResource(ClassLoader.java:1418) at org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2861) at org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3135) at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:3094) at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:3067) at org.apache.hadoop.conf.Configuration.loadProps(Configuration.java:2945) at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2927) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1265) at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1319) at org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1545) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.<clinit>(PacketReceiver.java:82) at org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.<init>(BlockReaderRemote.java:101) at org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.newBlockReader(BlockReaderRemote.java:437) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:861) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:757) at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:381) at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:715) at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:645) at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:845) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:919) at java.base/java.io.DataInputStream.read(DataInputStream.java:158) at java.base/java.io.InputStream.transferTo(InputStream.java:796) at java.base/java.nio.file.Files.copy(Files.java:3151) at java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:216) at java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:212) at java.base/java.security.AccessController.doPrivileged(AccessController.java:571) at java.base/sun.net.www.protocol.jar.URLJarFile.retrieve(URLJarFile.java:211) at java.base/sun.net.www.protocol.jar.URLJarFile.getJarFile(URLJarFile.java:71) at java.base/sun.net.www.protocol.jar.JarFileFactory.get(JarFileFactory.java:153) at java.base/sun.net.www.protocol.jar.JarURLConnection.connect(JarURLConnection.java:109) at java.base/sun.net.www.protocol.jar.JarURLConnection.getJarFile(JarURLConnection.java:70) at java.base/jdk.internal.loader.URLClassPath$JarLoader.getJarFile(URLClassPath.java:814) at java.base/jdk.internal.loader.URLClassPath$JarLoader$1.run(URLClassPath.java:774) at java.base/jdk.internal.loader.URLClassPath$JarLoader$1.run(URLClassPath.java:768) at java.base/java.security.AccessController.doPrivileged(AccessController.java:714) at java.base/jdk.internal.loader.URLClassPath$JarLoader.ensureOpen(URLClassPath.java:767) at java.base/jdk.internal.loader.URLClassPath$JarLoader.<init>(URLClassPath.java:734) at java.base/jdk.internal.loader.URLClassPath$3.run(URLClassPath.java:497) at java.base/jdk.internal.loader.URLClassPath$3.run(URLClassPath.java:479) at java.base/java.security.AccessController.doPrivileged(AccessController.java:714) at java.base/jdk.internal.loader.URLClassPath.getLoader(URLClassPath.java:478) at java.base/jdk.internal.loader.URLClassPath.getLoader(URLClassPath.java:446) at java.base/jdk.internal.loader.URLClassPath.findResource(URLClassPath.java:292) at java.base/java.net.URLClassLoader$2.run(URLClassLoader.java:629) at java.base/java.net.URLClassLoader$2.run(URLClassLoader.java:627) at java.base/java.security.AccessController.doPrivileged(AccessController.java:400) at java.base/java.net.URLClassLoader.findResource(URLClassLoader.java:626) at java.base/java.lang.ClassLoader.getResource(ClassLoader.java:1418) at org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2861) at org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3135) at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:3094) at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:3067) at org.apache.hadoop.conf.Configuration.loadProps(Configuration.java:2945) at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2927) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1265) at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1319) at org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:1726) at org.apache.hadoop.fs.statistics.impl.IOStatisticsContextIntegration.<clinit>(IOStatisticsContextIntegration.java:79) at org.apache.hadoop.fs.statistics.IOStatisticsContext.getCurrentIOStatisticsContext(IOStatisticsContext.java:75) at org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.<init>(RawLocalFileSystem.java:173) at org.apache.hadoop.fs.RawLocalFileSystem.open(RawLocalFileSystem.java:393) at org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>(ChecksumFileSystem.java:189) at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:581) at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:995) at org.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:489) at org.apache.hadoop.hive.cli.CliDriver.processInitFiles(CliDriver.java:524) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:204) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:75) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:52) at java.base/java.lang.reflect.Method.invoke(Method.java:580) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1031) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:226) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:100) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1136) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1145) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) ``` </details> > Avoid PacketReceiver#MAX_PACKET_SIZE Initialized to 0 > ----------------------------------------------------- > > Key: HDFS-17630 > URL: https://issues.apache.org/jira/browse/HDFS-17630 > Project: Hadoop HDFS > Issue Type: Bug > Affects Versions: 3.4.0 > Reporter: dzcxzl > Priority: Major > Labels: pull-request-available > > There are nested calls, causing the MAX_PACKET_SIZE of PacketReceiver to be 0. > > {code:java} > java.io.IOException: Incorrect value for packet payload size: 1014776 > at > org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:167) > at > org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:112) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.readNextPacket(BlockReaderRemote.java:187) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.read(BlockReaderRemote.java:146) > at > org.apache.hadoop.hdfs.ByteArrayStrategy.readFromBlock(ReaderStrategy.java:118) > at > org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:789) > at > org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:855) > at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:919) > at java.base/java.io.DataInputStream.read(DataInputStream.java:158) > at java.base/java.io.InputStream.transferTo(InputStream.java:796) > at java.base/java.nio.file.Files.copy(Files.java:3151) > at > java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:216) > at > java.base/sun.net.www.protocol.jar.URLJarFile$1.run(URLJarFile.java:212) > at > java.base/java.security.AccessController.doPrivileged(AccessController.java:571) > at > org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1319) > at org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1545) > at > org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.<clinit>(PacketReceiver.java:82) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.<init>(BlockReaderRemote.java:101) > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org