-
Notifications
You must be signed in to change notification settings - Fork 8.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HDFS-17630. Avoid PacketReceiver#MAX_PACKET_SIZE Initialized to 0 #7063
base: trunk
Are you sure you want to change the base?
Conversation
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
Spark uses FsUrlStreamHandlerFactory to support HDFS Jar, but in some scenarios PacketReceiver will be called nested, causing Spark to fail to start. cc @sunchao PacketReceiver Exceptionjava.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) |
Hadoop 3.4.1 is being released, Spark may not work properly with Hadoop 3.4.x. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
style review. Someone who knows HDFS code must review this -not me
...doop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/TestPacketReceiver.java
Outdated
Show resolved
Hide resolved
...p-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
Outdated
Show resolved
Hide resolved
I don't normally go near hdfs, so had missed this. It also means: I don't review their patches. I think this should target 3.4.2 and we focus on getting that out |
I also agree with Steve's comment. +1 for targeting this for 3.4.2 instead of the on-going 3.4.1 RC3 vote. |
💔 -1 overall
This message was automatically generated. |
...p-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
Outdated
Show resolved
Hide resolved
int totalLen = payloadLen + headerLen; | ||
if (totalLen < 0 || totalLen > MAX_PACKET_SIZE) { | ||
if (totalLen < 0 || totalLen > maxPacketSize) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we check this static final variable, MAX_PACKET_SIZE
, at this runtime layer instead of the initialization ?
Lines 80 to 85 in 9a743bd
static { | |
Configuration conf = new HdfsConfiguration(); | |
MAX_PACKET_SIZE = conf.getInt(HdfsClientConfigKeys. | |
DFS_DATA_TRANSFER_MAX_PACKET_SIZE, | |
HdfsClientConfigKeys.DFS_DATA_TRANSFER_MAX_PACKET_SIZE_DEFAULT); | |
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
PacketReceiver may make nested calls when reading conf, and the static block has not yet been initialized.
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.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)
💔 -1 overall
This message was automatically generated. |
Description of PR
There are nested calls, causing the MAX_PACKET_SIZE of PacketReceiver to be 0.
Related HDFS-15469
How was this patch tested?
For code changes:
LICENSE
,LICENSE-binary
,NOTICE-binary
files?