[
https://issues.apache.org/jira/browse/HUDI-3744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17642357#comment-17642357
]
Cosmin Iordache edited comment on HUDI-3744 at 12/2/22 3:49 PM:
----------------------------------------------------------------
I have the same error:
*Description of the issue follows:* * I'm using {{hudi-spark3-bundle-2.12}} on
{{spark=3.3.1}} with {{hadoop=3.3.4}} .
* The error happens on a COW table when we do a couple of ingestions -
apparently the {{view}} is not synchronized with the client . So at this place
: {{RequestHandler.syncIfLocalViewBehind(RequestHandler.java:163)}}
*What happens:*
The reason is that {{FSDataInputStreamWrapper}} that is shaded in hudi comes
from {{hbase=2.4.9}} this hbase that is used for hudi compilation is itself
compiled against {{hadoop=2.10.1}} which has the following definition for the
method {{getReadStatistics()}}
[https://github.com/apache/hadoop/blob/branch-2.10/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java#L106]
While in {{hadoop=3.3.4}} the definition is :
[https://github.com/apache/hadoop/blob/branch-3.3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java#L107]
The code in the {{FSDataInputStreamWrapper}} directly references the
{{hadoop=2.10}} version of the {{getReadStatistics()}} :
This is a reference to {{DFSInputStream$ReadStatistics}} in the decompiled
{{FSDataInputStreamWrapper}} that comes with {{hudi-spark3-bundle-2.12}}
#203 = Utf8 getReadStatistics
#204 = Utf8 ()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
#205 = NameAndType #203:#204 //
getReadStatistics:()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
#206 = Methodref #197.#205 //
org/apache/hadoop/hdfs/client/HdfsDataInputStream.getReadStatistics:()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
Stacktrace:
{code:java}
2022-12-01/14:57:52.770 [qtp700018681-1038] INFO
o.a.h.c.t.t.HoodieActiveTimeline - Loaded instants upto :
Option{val=[20221201145745641__deltacommit__COMPLETED]}2022-12-01/14:57:52.770
[qtp700018681-1038] DEBUG o.a.h.t.service.RequestHandler -
TimeTakenMillis[Total=7, Refresh=0, handle=0, Check=0], Success=true,
Query=basepath=hdfs%3A%2F%2Fnamenode%3A8020%2Fdata%2Flake%2F20221201-145418-342%2F1669902858975_112%2Fconverted&lastinstantts=20221201145745641&timelinehash=9a4be98105ed813c7483792025cface70d7d7d51ddeef6f7565d119eb176da9a,
Host=192.168.15.158:38537, synced=false2022-12-01/14:57:52.770
[qtp700018681-1038] ERROR io.javalin.Javalin - Exception occurred while
servicing http-requestjava.lang.NoSuchMethodError:
'org.apache.hadoop.hdfs.DFSInputStream$ReadStatistics
org.apache.hadoop.hdfs.client.HdfsDataInputStream.getReadStatistics()'
at
org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.updateInputStreamStatistics(FSDataInputStreamWrapper.java:249)
at
org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.close(FSDataInputStreamWrapper.java:296)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.closeStreams(HFileBlock.java:1825)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFilePreadReader.close(HFilePreadReader.java:107)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.close(HFileReaderImpl.java:1421)
at
org.apache.hudi.io.storage.HoodieHFileReader.close(HoodieHFileReader.java:218)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.closeReader(HoodieBackedTableMetadata.java:589)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.close(HoodieBackedTableMetadata.java:572)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.closePartitionReaders(HoodieBackedTableMetadata.java:580)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.reset(HoodieBackedTableMetadata.java:647)
at
org.apache.hudi.metadata.HoodieMetadataFileSystemView.sync(HoodieMetadataFileSystemView.java:77)
at
org.apache.hudi.timeline.service.RequestHandler.syncIfLocalViewBehind(RequestHandler.java:163)
at
org.apache.hudi.timeline.service.RequestHandler.access$100(RequestHandler.java:68)
at
org.apache.hudi.timeline.service.RequestHandler$ViewHandler.handle(RequestHandler.java:494)
at
io.javalin.security.SecurityUtil.noopAccessManager(SecurityUtil.kt:22)
at io.javalin.Javalin.lambda$addHandler$0(Javalin.java:606) at
io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:46)
at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:17)
at io.javalin.core.JavalinServlet$service$1.invoke(JavalinServlet.kt:143)
at io.javalin.core.JavalinServlet$service$2.invoke(JavalinServlet.kt:41)
at io.javalin.core.JavalinServlet.service(JavalinServlet.kt:107) at
io.javalin.core.util.JettyServerUtil$initialize$httpHandler$1.doHandle(JettyServerUtil.kt:72)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
at
org.apache.hudi.org.apache.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
at
org.apache.hudi.org.apache.jetty.server.session.SessionHandler.doScope(SessionHandler.java:1668)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
at
org.apache.hudi.org.apache.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
at
org.apache.hudi.org.apache.jetty.server.handler.HandlerList.handle(HandlerList.java:61)
at
org.apache.hudi.org.apache.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:174)
at
org.apache.hudi.org.apache.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
at
org.apache.hudi.org.apache.jetty.server.Server.handle(Server.java:502)
at
org.apache.hudi.org.apache.jetty.server.HttpChannel.handle(HttpChannel.java:370)
at
org.apache.hudi.org.apache.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
at
org.apache.hudi.org.apache.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
at
org.apache.hudi.org.apache.jetty.io.FillInterest.fillable(FillInterest.java:103)
at
org.apache.hudi.org.apache.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:333)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:310)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:168)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.produce(EatWhatYouKill.java:132)
at
org.apache.hudi.org.apache.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:765)
at
org.apache.hudi.org.apache.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:683)
at java.base/java.lang.Thread.run(Thread.java:829) {code}
Update :
Managed to fix the problem on my local by using an hbase version that was
compiled using `-Dhadoop.profile=3.0` .
Then I had to also upgrade commons.io to 2.11 in hudi and now I have a working
version.
was (Author: arw357):
I have the same error:
*Description of the issue follows:* * I'm using {{hudi-spark3-bundle-2.12}} on
{{spark=3.3.1}} with {{hadoop=3.3.4}} .
* The error happens on a COW table when we do a couple of ingestions -
apparently the {{view}} is not synchronized with the client . So at this place
: {{RequestHandler.syncIfLocalViewBehind(RequestHandler.java:163)}}
*What happens:*
The reason is that {{FSDataInputStreamWrapper}} that is shaded in hudi comes
from {{hbase=2.4.9}} this hbase that is used for hudi compilation is itself
compiled against {{hadoop=2.10.1}} which has the following definition for the
method {{getReadStatistics()}}
[https://github.com/apache/hadoop/blob/branch-2.10/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java#L106]
While in {{hadoop=3.3.4}} the definition is :
[https://github.com/apache/hadoop/blob/branch-3.3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java#L107]
The code in the {{FSDataInputStreamWrapper}} directly references the
{{hadoop=2.10}} version of the {{getReadStatistics()}} :
This is a reference to {{DFSInputStream$ReadStatistics}} in the decompiled
{{FSDataInputStreamWrapper}} that comes with {{hudi-spark3-bundle-2.12}}
#203 = Utf8 getReadStatistics
#204 = Utf8
()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
#205 = NameAndType #203:#204 //
getReadStatistics:()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
#206 = Methodref #197.#205 //
org/apache/hadoop/hdfs/client/HdfsDataInputStream.getReadStatistics:()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
Stacktrace:
{code:java}
2022-12-01/14:57:52.770 [qtp700018681-1038] INFO
o.a.h.c.t.t.HoodieActiveTimeline - Loaded instants upto :
Option{val=[20221201145745641__deltacommit__COMPLETED]}2022-12-01/14:57:52.770
[qtp700018681-1038] DEBUG o.a.h.t.service.RequestHandler -
TimeTakenMillis[Total=7, Refresh=0, handle=0, Check=0], Success=true,
Query=basepath=hdfs%3A%2F%2Fnamenode%3A8020%2Fdata%2Flake%2F20221201-145418-342%2F1669902858975_112%2Fconverted&lastinstantts=20221201145745641&timelinehash=9a4be98105ed813c7483792025cface70d7d7d51ddeef6f7565d119eb176da9a,
Host=192.168.15.158:38537, synced=false2022-12-01/14:57:52.770
[qtp700018681-1038] ERROR io.javalin.Javalin - Exception occurred while
servicing http-requestjava.lang.NoSuchMethodError:
'org.apache.hadoop.hdfs.DFSInputStream$ReadStatistics
org.apache.hadoop.hdfs.client.HdfsDataInputStream.getReadStatistics()'
at
org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.updateInputStreamStatistics(FSDataInputStreamWrapper.java:249)
at
org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.close(FSDataInputStreamWrapper.java:296)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.closeStreams(HFileBlock.java:1825)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFilePreadReader.close(HFilePreadReader.java:107)
at
org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.close(HFileReaderImpl.java:1421)
at
org.apache.hudi.io.storage.HoodieHFileReader.close(HoodieHFileReader.java:218)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.closeReader(HoodieBackedTableMetadata.java:589)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.close(HoodieBackedTableMetadata.java:572)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.closePartitionReaders(HoodieBackedTableMetadata.java:580)
at
org.apache.hudi.metadata.HoodieBackedTableMetadata.reset(HoodieBackedTableMetadata.java:647)
at
org.apache.hudi.metadata.HoodieMetadataFileSystemView.sync(HoodieMetadataFileSystemView.java:77)
at
org.apache.hudi.timeline.service.RequestHandler.syncIfLocalViewBehind(RequestHandler.java:163)
at
org.apache.hudi.timeline.service.RequestHandler.access$100(RequestHandler.java:68)
at
org.apache.hudi.timeline.service.RequestHandler$ViewHandler.handle(RequestHandler.java:494)
at
io.javalin.security.SecurityUtil.noopAccessManager(SecurityUtil.kt:22)
at io.javalin.Javalin.lambda$addHandler$0(Javalin.java:606) at
io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:46)
at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:17)
at io.javalin.core.JavalinServlet$service$1.invoke(JavalinServlet.kt:143)
at io.javalin.core.JavalinServlet$service$2.invoke(JavalinServlet.kt:41)
at io.javalin.core.JavalinServlet.service(JavalinServlet.kt:107) at
io.javalin.core.util.JettyServerUtil$initialize$httpHandler$1.doHandle(JettyServerUtil.kt:72)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
at
org.apache.hudi.org.apache.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
at
org.apache.hudi.org.apache.jetty.server.session.SessionHandler.doScope(SessionHandler.java:1668)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
at
org.apache.hudi.org.apache.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
at
org.apache.hudi.org.apache.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
at
org.apache.hudi.org.apache.jetty.server.handler.HandlerList.handle(HandlerList.java:61)
at
org.apache.hudi.org.apache.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:174)
at
org.apache.hudi.org.apache.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
at
org.apache.hudi.org.apache.jetty.server.Server.handle(Server.java:502)
at
org.apache.hudi.org.apache.jetty.server.HttpChannel.handle(HttpChannel.java:370)
at
org.apache.hudi.org.apache.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
at
org.apache.hudi.org.apache.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
at
org.apache.hudi.org.apache.jetty.io.FillInterest.fillable(FillInterest.java:103)
at
org.apache.hudi.org.apache.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:333)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:310)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:168)
at
org.apache.hudi.org.apache.jetty.util.thread.strategy.EatWhatYouKill.produce(EatWhatYouKill.java:132)
at
org.apache.hudi.org.apache.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:765)
at
org.apache.hudi.org.apache.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:683)
at java.base/java.lang.Thread.run(Thread.java:829) {code}
> NoSuchMethodError of getReadStatistics with Spark 3.2/Hadoop 3.2 using HBase
> -----------------------------------------------------------------------------
>
> Key: HUDI-3744
> URL: https://issues.apache.org/jira/browse/HUDI-3744
> Project: Apache Hudi
> Issue Type: Bug
> Reporter: Ethan Guo
> Assignee: Raymond Xu
> Priority: Blocker
> Fix For: 0.11.0
>
>
> Environment: Hadoop 3.2.1 & Spark-3.2.1
> hudi compile from commit f2a93ead3b5a6964a72b3543ada58aa334edef9c
> just use spark-sql and default job configuration to execute "show partitions
> [hudi_table_name];"
> {code:java}
> // command
> spark-sql --conf spark.serializer=org.apache.spark.serializer.KryoSerializer
> --conf
> spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension
> --conf
> spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog
> // spark-sql
> spark-sql> show partitions hudi_partition_table;
> {code}
> // code placeholderjava.lang.NoSuchMethodError:
> org.apache.hadoop.hdfs.client.HdfsDataInputStream.getReadStatistics()Lorg/apache/hadoop/hdfs/DFSInputStream$ReadStatistics;
> at
> org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.updateInputStreamStatistics(FSDataInputStreamWrapper.java:249)
> at
> org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.close(FSDataInputStreamWrapper.java:296)
> at
> org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.closeStreams(HFileBlock.java:1825)
> at
> org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFilePreadReader.close(HFilePreadReader.java:107)
> at
> org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.close(HFileReaderImpl.java:1421)
> at
> org.apache.hudi.io.storage.HoodieHFileReader.close(HoodieHFileReader.java:423)
> at
> org.apache.hudi.metadata.HoodieBackedTableMetadata.close(HoodieBackedTableMetadata.java:435)
> at
> org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$0(HoodieBackedTableMetadata.java:162)
> at java.util.HashMap.forEach(HashMap.java:1290)
> at
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:138)
> at
> org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:128)
> at
> org.apache.hudi.metadata.BaseTableMetadata.fetchAllPartitionPaths(BaseTableMetadata.java:281)
> at
> org.apache.hudi.metadata.BaseTableMetadata.getAllPartitionPaths(BaseTableMetadata.java:111)
> at
> org.apache.hudi.common.fs.FSUtils.getAllPartitionPaths(FSUtils.java:308)
> at
> org.apache.spark.sql.hudi.HoodieSqlCommonUtils$.getAllPartitionPaths(HoodieSqlCommonUtils.scala:81)
> at
> org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.getPartitionPaths(HoodieCatalogTable.scala:157)
> at
> org.apache.spark.sql.hudi.command.ShowHoodieTablePartitionsCommand.run(ShowHoodieTablePartitionsCommand.scala:51)
> at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
> at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
> at
> org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
> at
> org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:110)
> at
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
> at
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
> at
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
> at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
> at
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
> at
> org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
> at
> org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:106)
--
This message was sent by Atlassian Jira
(v8.20.10#820010)