You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/24 07:27:54 UTC

[GitHub] [hudi] xiaokxluoshu opened a new issue, #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

xiaokxluoshu opened a new issue, #7046:
URL: https://github.com/apache/hudi/issues/7046

   
   **Environment Description**
   
   * Hudi version : 0.12.0
   
   * Spark version : 2.4
   
   * Flink version: 1.13.6
   
   * Hive version : 
   
   * Hadoop version : 3.2.2
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Running on Docker? (yes/no) : NO
   
   
   **Describe the problem you faced**
   We use flink to write hudi and use table.type=cow to write 20 million data and then stop. After 30 minutes, we write again
   ````
   2022-10-24 11:43:52,599 INFO  org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner [] - MaxMemoryInBytes allowed for compaction => 1073741824
   2022-10-24 11:43:52,599 INFO  org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner [] - Number of entries in MemoryBasedMap in ExternalSpillableMap => 2
   2022-10-24 11:43:52,599 INFO  org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner [] - Total size in bytes of MemoryBasedMap in ExternalSpillableMap => 8656
   2022-10-24 11:43:52,599 INFO  org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner [] - Number of entries in BitCaskDiskMap in ExternalSpillableMap => 0
   2022-10-24 11:43:52,599 INFO  org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner [] - Size of file spilled to disk => 0
   2022-10-24 11:43:52,600 INFO  org.apache.hudi.io.HoodieMergeHandle                         [] - partitionPath:files, fileId to be merged:files-0000
   2022-10-24 11:43:52,603 INFO  org.apache.hudi.io.HoodieMergeHandle                         [] - Merging new data into oldPath hdfs://hadoop01:9000/hudi/model-series-intact-data/.hoodie/metadata/files/files-0000_0-1-0_20221024100338853001.hfile, as newPath hdfs://hadoop01:9000/hudi/model-series-intact-data/.hoodie/metadata/files/files-0000_0-1-0_20221024114333232001.hfile
   2022-10-24 11:43:52,664 INFO  org.apache.hudi.table.marker.DirectWriteMarkers              [] - Creating Marker Path=hdfs://hadoop01:9000/hudi/model-series-intact-data/.hoodie/metadata/.hoodie/.temp/20221024114333232001/files/files-0000_0-1-0_20221024114333232001.hfile.marker.MERGE
   2022-10-24 11:43:52,674 INFO  org.apache.hudi.table.marker.DirectWriteMarkers              [] - [direct] Created marker file hdfs://hadoop01:9000/hudi/model-series-intact-data/.hoodie/metadata/.hoodie/.temp/20221024114333232001/files/files-0000_0-1-0_20221024114333232001.hfile.marker.MERGE in 71 ms
   2022-10-24 11:43:52,681 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new compressor [.gz]
   2022-10-24 11:43:52,682 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new compressor [.gz]
   2022-10-24 11:43:52,689 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new decompressor [.gz]
   2022-10-24 11:43:52,689 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new decompressor [.gz]
   2022-10-24 11:43:52,689 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new decompressor [.gz]
   2022-10-24 11:43:52,690 INFO  org.apache.hudi.common.util.queue.IteratorBasedQueueProducer [] - starting to buffer records
   2022-10-24 11:43:52,691 INFO  org.apache.hudi.common.util.queue.BoundedInMemoryExecutor    [] - starting consumer thread
   2022-10-24 11:43:52,695 INFO  org.apache.hadoop.io.compress.CodecPool                      [] - Got brand-new decompressor [.gz]
   2022-10-24 11:43:52,697 INFO  org.apache.hudi.common.util.queue.IteratorBasedQueueProducer [] - finished buffering records
   2022-10-24 11:43:52,699 INFO  org.apache.hudi.common.util.queue.BoundedInMemoryExecutor    [] - Queue Consumption is done; notifying producer threads
   2022-10-24 11:43:52,700 ERROR org.apache.hudi.sink.StreamWriteOperatorCoordinator          [] - Executor executes action [initialize instant 20221024114339630] error
   java.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) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.org.apache.hadoop.hbase.io.FSDataInputStreamWrapper.close(FSDataInputStreamWrapper.java:296) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.closeStreams(HFileBlock.java:1825) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFilePreadReader.close(HFilePreadReader.java:107) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.close(HFileReaderImpl.java:1421) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.io.storage.HoodieHFileReader.close(HoodieHFileReader.java:218) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.action.commit.FlinkMergeHelper.runMerge(FlinkMergeHelper.java:107) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdateInternal(HoodieFlinkCopyOnWriteTable.java:380) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdate(HoodieFlinkCopyOnWriteTable.java:371) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:227) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.lambda$compact$57154431$1(HoodieCompactor.java:138) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.common.function.FunctionWrapper.lambda$throwingMapWrapper$0(FunctionWrapper.java:38) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) ~[?:1.8.0_261]
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:747) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:721) ~[?:1.8.0_261]
   	at java.util.stream.AbstractTask.compute(AbstractTask.java:316) ~[?:1.8.0_261]
   	at java.util.concurrent.CountedCompleter.exec(CountedCompleter.java:731) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.doInvoke(ForkJoinTask.java:401) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.invoke(ForkJoinTask.java:734) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceOp.evaluateParallel(ReduceOps.java:714) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:233) ~[?:1.8.0_261]
   	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) ~[?:1.8.0_261]
   	at org.apache.hudi.common.data.HoodieBaseListData.<init>(HoodieBaseListData.java:41) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.common.data.HoodieListData.<init>(HoodieListData.java:68) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.common.data.HoodieListData.map(HoodieListData.java:105) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:138) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.action.compact.RunCompactionActionExecutor.execute(RunCompactionActionExecutor.java:88) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.table.HoodieFlinkMergeOnReadTable.compact(HoodieFlinkMergeOnReadTable.java:118) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.compact(HoodieFlinkWriteClient.java:429) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.BaseHoodieWriteClient.lambda$runAnyPendingCompactions$1(BaseHoodieWriteClient.java:592) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384) ~[?:1.8.0_261]
   	at java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:580) ~[?:1.8.0_261]
   	at org.apache.hudi.client.BaseHoodieWriteClient.runAnyPendingCompactions(BaseHoodieWriteClient.java:590) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.BaseHoodieWriteClient.runAnyPendingCompactions(BaseHoodieWriteClient.java:641) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.compactIfNecessary(HoodieBackedTableMetadataWriter.java:1007) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter.commit(FlinkHoodieBackedTableMetadataWriter.java:115) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.processAndCommit(HoodieBackedTableMetadataWriter.java:813) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:880) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.writeTableMetadata(HoodieFlinkWriteClient.java:281) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:271) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:236) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:117) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:530) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:506) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:477) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$initInstant$6(StreamWriteOperatorCoordinator.java:399) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130) ~[blob_p-85bf6a9f06cb91bd20775cc5cfd6a95060810396-32878570929da3966468820ac5787a7f:0.12.0]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_261]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_261]
   	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_261]
   2022-10-24 11:43:52,703 INFO  org.apache.flink.runtime.jobmaster.JobMaster                 [] - Trying to recover from a global failure.
   org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'hoodie_append_write: data_hudi_sink' (operator 20600d0a6cfd8abe54715a1cd220ef3f).
   	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:553) ~[flink-dist_2.11-1.13.6.jar:1.13.6]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$0(StreamWriteOperatorCoordinator.java:187) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.handleException(NonThrownExecutor.java:146) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:133) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_261]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_261]
   	at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_261]
   Caused by: org.apache.hudi.exception.HoodieException: Executor executes action [initialize instant 20221024114339630] error
   	... 6 more
   Caused by: java.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:218) ~[?:?]
   	at org.apache.hudi.table.action.commit.FlinkMergeHelper.runMerge(FlinkMergeHelper.java:107) ~[?:?]
   	at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdateInternal(HoodieFlinkCopyOnWriteTable.java:380) ~[?:?]
   	at org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleUpdate(HoodieFlinkCopyOnWriteTable.java:371) ~[?:?]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:227) ~[?:?]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.lambda$compact$57154431$1(HoodieCompactor.java:138) ~[?:?]
   	at org.apache.hudi.common.function.FunctionWrapper.lambda$throwingMapWrapper$0(FunctionWrapper.java:38) ~[?:?]
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) ~[?:1.8.0_261]
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:747) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:721) ~[?:1.8.0_261]
   	at java.util.stream.AbstractTask.compute(AbstractTask.java:316) ~[?:1.8.0_261]
   	at java.util.concurrent.CountedCompleter.exec(CountedCompleter.java:731) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.doInvoke(ForkJoinTask.java:401) ~[?:1.8.0_261]
   	at java.util.concurrent.ForkJoinTask.invoke(ForkJoinTask.java:734) ~[?:1.8.0_261]
   	at java.util.stream.ReduceOps$ReduceOp.evaluateParallel(ReduceOps.java:714) ~[?:1.8.0_261]
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:233) ~[?:1.8.0_261]
   	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) ~[?:1.8.0_261]
   	at org.apache.hudi.common.data.HoodieBaseListData.<init>(HoodieBaseListData.java:41) ~[?:?]
   	at org.apache.hudi.common.data.HoodieListData.<init>(HoodieListData.java:68) ~[?:?]
   	at org.apache.hudi.common.data.HoodieListData.map(HoodieListData.java:105) ~[?:?]
   	at org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:138) ~[?:?]
   	at org.apache.hudi.table.action.compact.RunCompactionActionExecutor.execute(RunCompactionActionExecutor.java:88) ~[?:?]
   	at org.apache.hudi.table.HoodieFlinkMergeOnReadTable.compact(HoodieFlinkMergeOnReadTable.java:118) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.compact(HoodieFlinkWriteClient.java:429) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.lambda$runAnyPendingCompactions$1(BaseHoodieWriteClient.java:592) ~[?:?]
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384) ~[?:1.8.0_261]
   	at java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:580) ~[?:1.8.0_261]
   	at org.apache.hudi.client.BaseHoodieWriteClient.runAnyPendingCompactions(BaseHoodieWriteClient.java:590) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.runAnyPendingCompactions(BaseHoodieWriteClient.java:641) ~[?:?]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.compactIfNecessary(HoodieBackedTableMetadataWriter.java:1007) ~[?:?]
   	at org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter.commit(FlinkHoodieBackedTableMetadataWriter.java:115) ~[?:?]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.processAndCommit(HoodieBackedTableMetadataWriter.java:813) ~[?:?]
   	at org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.update(HoodieBackedTableMetadataWriter.java:880) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.writeTableMetadata(HoodieFlinkWriteClient.java:281) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:271) ~[?:?]
   	at org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:236) ~[?:?]
   	at org.apache.hudi.client.HoodieFlinkWriteClient.commit(HoodieFlinkWriteClient.java:117) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.doCommit(StreamWriteOperatorCoordinator.java:530) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:506) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.commitInstant(StreamWriteOperatorCoordinator.java:477) ~[?:?]
   	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$initInstant$6(StreamWriteOperatorCoordinator.java:399) ~[?:?]
   	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130) ~[?:?]
   	... 3 more
   ````
   
   I found that the hadoop version used in hudi source code is 2.10.1, resulting in the hadoop-hdfs-client version is 2.10.1, However, I use hadoop 3.2.2 and the version of hadoop-hdfs-client is 3.2.2 in the flink automatic import package `hadoop classpath`. In this case, the above exception will occur. Is there any solution?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xiaokxluoshu commented on issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

Posted by GitBox <gi...@apache.org>.
xiaokxluoshu commented on issue #7046:
URL: https://github.com/apache/hudi/issues/7046#issuecomment-1288544166

   There is one more condition for this exception: response status 500 error occurs when a request is made to hadoop. Everything else will work. FSDataInputStreamWrapper updateInputStreamStatistics can add exception handling.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #7046:
URL: https://github.com/apache/hudi/issues/7046#issuecomment-1289732461

   Ref: https://github.com/apache/hudi/issues/5765
   
   feel free to re-open is you are still facing issues. thanks! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan closed issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many  java.lang.NoSuchMethodError
URL: https://github.com/apache/hudi/issues/7046


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #7046:
URL: https://github.com/apache/hudi/issues/7046#issuecomment-1289703643

   May be this [FAQ](https://github.com/apache/hudi/issues/7046) could help you. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on issue #7046: [SUPPORT] hadoop use 3.2.2 ,There are so many java.lang.NoSuchMethodError

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #7046:
URL: https://github.com/apache/hudi/issues/7046#issuecomment-1306009653

   https://hudi.apache.org/docs/faq#how-can-i-resolve-the-nosuchmethoderror-from-hbase-when-using-hudi-with-metadata-table-on-hdfs


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org