[ https://issues.apache.org/jira/browse/FLINK-35724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
zhi.wang updated FLINK-35724: ----------------------------- Description: writing data to oss with jdk11 will throw an exception of `java.lang.NoClassDefFoundError: javax/xml/bind/JAXBException` — bug conditions: * jdk11: using this image with docker.io/library/flink:1.19-java11 which contains jdk11 * enable plugin flink-s3-fs-hadoop-1.19.*.jar * there's an error in oss connections, for example, access secret key error — exceptions caught in taskmanager: {quote}2024-06-28 19:07:01,577 WARN org.apache.flink.runtime.taskmanager.Task [] - Source: data-generator-source -> Map -> Sink: Writer -> Sink: Committer (2/2)#6 (2d837c57492b8c19ea522293b149d7b9_cbc357ccb763df2852fee8c4fc7d55f2_1_6) switched from RUNNING to FAILED with failure cause: java.lang.NoClassDefFoundError: javax/xml/bind/JAXBException at com.aliyun.oss.internal.OSSErrorResponseHandler.handle(OSSErrorResponseHandler.java:69) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.handleResponse(ServiceClient.java:259) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.sendRequestImpl(ServiceClient.java:142) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.sendRequest(ServiceClient.java:71) ~[?:?] at com.aliyun.oss.internal.OSSOperation.send(OSSOperation.java:86) ~[?:?] at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:148) ~[?:?] at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:105) ~[?:?] at com.aliyun.oss.internal.OSSMultipartOperation.initiateMultipartUpload(OSSMultipartOperation.java:249) ~[?:?] at com.aliyun.oss.OSSClient.initiateMultipartUpload(OSSClient.java:866) ~[?:?] at org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.getUploadId(AliyunOSSFileSystemStore.java:699) ~[?:?] at org.apache.flink.fs.osshadoop.OSSAccessor.startMultipartUpload(OSSAccessor.java:61) ~[?:?] at org.apache.flink.fs.osshadoop.writer.OSSRecoverableMultipartUpload.<init>(OSSRecoverableMultipartUpload.java:81) ~[?:?] at org.apache.flink.fs.osshadoop.writer.OSSRecoverableWriter.open(OSSRecoverableWriter.java:73) ~[?:?] at org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter$OutputStreamBasedBucketWriter.openNewInProgressFile(OutputStreamBasedPartFileWriter.java:124) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.functions.sink.filesystem.BulkBucketWriter.openNewInProgressFile(BulkBucketWriter.java:36) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriterBucket.rollPartFile(FileWriterBucket.java:261) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriterBucket.write(FileWriterBucket.java:188) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriter.write(FileWriter.java:198) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator.processElement(SinkWriterOperator.java:160) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:38) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask$AsyncDataOutputToOutput.emitRecord(SourceOperatorStreamTask.java:310) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:110) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:101) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.api.connector.source.lib.util.IteratorSourceReaderBase.pollNext(IteratorSourceReaderBase.java:111) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader.pollNext(RateLimitedSourceReader.java:69) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:421) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:579) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:909) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:858) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) [flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:751) [flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) [flink-dist-1.19.0.jar:1.19.0] at java.lang.Thread.run(Unknown Source) [?:?] Caused by: java.lang.ClassNotFoundException: javax.xml.bind.JAXBException at java.net.URLClassLoader.findClass(Unknown Source) ~[?:?] at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] at org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentOnly(ComponentClassLoader.java:150) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:113) ~[flink-dist-1.19.0.jar:1.19.0] at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] ... 43 more {quote} — bug found: * [The JAXB APIs completely removed from the JDK 11]([https://stackoverflow.com/questions/43574426/how-to-resolve-java-lang-noclassdeffounderror-javax-xml-bind-jaxbexception)] * when error occurs while connecting to oss, OSSErrorResponseHandler will be accessed and then the exception will be thrown — how to fix: * oss sdk has already "Remove the parser of JAXBResponseParser" with this commit [https://github.com/aliyun/aliyun-oss-java-sdk/commit/c7b567c40c3a31b9b08092f5a49b567eaf0f854b] * therefore, just upgrade oss sdk version(the latest version is 3.17.4) — TODOs: * I have fixed and tested locally(with 1.19) * but I am confusing about the content of file `flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE` ** can it be generated automatically? was: writing data to oss with jdk11 will throw an exception of --- bug conditions: * jdk11: using this image with docker.io/library/flink:1.19-java11 which contains jdk11 * enable plugin flink-s3-fs-hadoop-1.19.*.jar * there's an error in oss connections, for example, access secret key error --- exceptions caught in taskmanager: {quote}2024-06-28 19:07:01,577 WARN org.apache.flink.runtime.taskmanager.Task [] - Source: data-generator-source -> Map -> Sink: Writer -> Sink: Committer (2/2)#6 (2d837c57492b8c19ea522293b149d7b9_cbc357ccb763df2852fee8c4fc7d55f2_1_6) switched from RUNNING to FAILED with failure cause: java.lang.NoClassDefFoundError: javax/xml/bind/JAXBException at com.aliyun.oss.internal.OSSErrorResponseHandler.handle(OSSErrorResponseHandler.java:69) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.handleResponse(ServiceClient.java:259) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.sendRequestImpl(ServiceClient.java:142) ~[?:?] at com.aliyun.oss.common.comm.ServiceClient.sendRequest(ServiceClient.java:71) ~[?:?] at com.aliyun.oss.internal.OSSOperation.send(OSSOperation.java:86) ~[?:?] at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:148) ~[?:?] at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:105) ~[?:?] at com.aliyun.oss.internal.OSSMultipartOperation.initiateMultipartUpload(OSSMultipartOperation.java:249) ~[?:?] at com.aliyun.oss.OSSClient.initiateMultipartUpload(OSSClient.java:866) ~[?:?] at org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.getUploadId(AliyunOSSFileSystemStore.java:699) ~[?:?] at org.apache.flink.fs.osshadoop.OSSAccessor.startMultipartUpload(OSSAccessor.java:61) ~[?:?] at org.apache.flink.fs.osshadoop.writer.OSSRecoverableMultipartUpload.<init>(OSSRecoverableMultipartUpload.java:81) ~[?:?] at org.apache.flink.fs.osshadoop.writer.OSSRecoverableWriter.open(OSSRecoverableWriter.java:73) ~[?:?] at org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter$OutputStreamBasedBucketWriter.openNewInProgressFile(OutputStreamBasedPartFileWriter.java:124) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.functions.sink.filesystem.BulkBucketWriter.openNewInProgressFile(BulkBucketWriter.java:36) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriterBucket.rollPartFile(FileWriterBucket.java:261) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriterBucket.write(FileWriterBucket.java:188) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.connector.file.sink.writer.FileWriter.write(FileWriter.java:198) ~[flink-connector-files-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator.processElement(SinkWriterOperator.java:160) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:38) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask$AsyncDataOutputToOutput.emitRecord(SourceOperatorStreamTask.java:310) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:110) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:101) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.api.connector.source.lib.util.IteratorSourceReaderBase.pollNext(IteratorSourceReaderBase.java:111) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader.pollNext(RateLimitedSourceReader.java:69) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:421) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:579) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:909) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:858) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) [flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:751) [flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) [flink-dist-1.19.0.jar:1.19.0] at java.lang.Thread.run(Unknown Source) [?:?] Caused by: java.lang.ClassNotFoundException: javax.xml.bind.JAXBException at java.net.URLClassLoader.findClass(Unknown Source) ~[?:?] at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] at org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentOnly(ComponentClassLoader.java:150) ~[flink-dist-1.19.0.jar:1.19.0] at org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:113) ~[flink-dist-1.19.0.jar:1.19.0] at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] ... 43 more {quote} --- bug found: * [The JAXB APIs completely removed from the JDK 11]([https://stackoverflow.com/questions/43574426/how-to-resolve-java-lang-noclassdeffounderror-javax-xml-bind-jaxbexception)] * when error occurs while connecting to oss, OSSErrorResponseHandler will be accessed and then the exception will be thrown --- how to fix: * oss sdk has already "Remove the parser of JAXBResponseParser" with this commit [https://github.com/aliyun/aliyun-oss-java-sdk/commit/c7b567c40c3a31b9b08092f5a49b567eaf0f854b] * therefore, just upgrade oss sdk version(the latest version is 3.17.4) --- TODOs: * I have fixed and tested locally(with 1.19) * but I am confusing about the content of file `flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE` ** can it be generated automatically? > javax.xml.bind.JAXBException not found with jdk11 while writing to oss > ---------------------------------------------------------------------- > > Key: FLINK-35724 > URL: https://issues.apache.org/jira/browse/FLINK-35724 > Project: Flink > Issue Type: Bug > Components: Connectors / FileSystem > Environment: flink version: 1.19 > jdk version: 11 > current oss provided by aliyun > with wrong access secret key to reproduce this exception > Reporter: zhi.wang > Priority: Major > > writing data to oss with jdk11 will throw an exception of > `java.lang.NoClassDefFoundError: javax/xml/bind/JAXBException` > — > bug conditions: > * jdk11: using this image with docker.io/library/flink:1.19-java11 which > contains jdk11 > * enable plugin flink-s3-fs-hadoop-1.19.*.jar > * there's an error in oss connections, for example, access secret key error > — > exceptions caught in taskmanager: > {quote}2024-06-28 19:07:01,577 WARN > org.apache.flink.runtime.taskmanager.Task [] - Source: > data-generator-source -> Map -> Sink: Writer -> Sink: Committer (2/2)#6 > (2d837c57492b8c19ea522293b149d7b9_cbc357ccb763df2852fee8c4fc7d55f2_1_6) > switched from RUNNING to FAILED with failure cause: > java.lang.NoClassDefFoundError: javax/xml/bind/JAXBException > at > com.aliyun.oss.internal.OSSErrorResponseHandler.handle(OSSErrorResponseHandler.java:69) > ~[?:?] > at > com.aliyun.oss.common.comm.ServiceClient.handleResponse(ServiceClient.java:259) > ~[?:?] > at > com.aliyun.oss.common.comm.ServiceClient.sendRequestImpl(ServiceClient.java:142) > ~[?:?] > at > com.aliyun.oss.common.comm.ServiceClient.sendRequest(ServiceClient.java:71) > ~[?:?] > at com.aliyun.oss.internal.OSSOperation.send(OSSOperation.java:86) > ~[?:?] > at > com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:148) ~[?:?] > at > com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:105) ~[?:?] > at > com.aliyun.oss.internal.OSSMultipartOperation.initiateMultipartUpload(OSSMultipartOperation.java:249) > ~[?:?] > at > com.aliyun.oss.OSSClient.initiateMultipartUpload(OSSClient.java:866) ~[?:?] > at > org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.getUploadId(AliyunOSSFileSystemStore.java:699) > ~[?:?] > at > org.apache.flink.fs.osshadoop.OSSAccessor.startMultipartUpload(OSSAccessor.java:61) > ~[?:?] > at > org.apache.flink.fs.osshadoop.writer.OSSRecoverableMultipartUpload.<init>(OSSRecoverableMultipartUpload.java:81) > ~[?:?] > at > org.apache.flink.fs.osshadoop.writer.OSSRecoverableWriter.open(OSSRecoverableWriter.java:73) > ~[?:?] > at > org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter$OutputStreamBasedBucketWriter.openNewInProgressFile(OutputStreamBasedPartFileWriter.java:124) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.api.functions.sink.filesystem.BulkBucketWriter.openNewInProgressFile(BulkBucketWriter.java:36) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.connector.file.sink.writer.FileWriterBucket.rollPartFile(FileWriterBucket.java:261) > ~[flink-connector-files-1.19.0.jar:1.19.0] > at > org.apache.flink.connector.file.sink.writer.FileWriterBucket.write(FileWriterBucket.java:188) > ~[flink-connector-files-1.19.0.jar:1.19.0] > at > org.apache.flink.connector.file.sink.writer.FileWriter.write(FileWriter.java:198) > ~[flink-connector-files-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator.processElement(SinkWriterOperator.java:160) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.api.operators.StreamMap.processElement(StreamMap.java:38) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.pushToOperator(CopyingChainingOutput.java:75) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:50) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.CopyingChainingOutput.collect(CopyingChainingOutput.java:29) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask$AsyncDataOutputToOutput.emitRecord(SourceOperatorStreamTask.java:310) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:110) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks.collect(SourceOutputWithWatermarks.java:101) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.api.connector.source.lib.util.IteratorSourceReaderBase.pollNext(IteratorSourceReaderBase.java:111) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader.pollNext(RateLimitedSourceReader.java:69) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:421) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:579) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:909) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:858) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) > [flink-dist-1.19.0.jar:1.19.0] > at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:751) > [flink-dist-1.19.0.jar:1.19.0] > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) > [flink-dist-1.19.0.jar:1.19.0] > at java.lang.Thread.run(Unknown Source) [?:?] > Caused by: java.lang.ClassNotFoundException: javax.xml.bind.JAXBException > at java.net.URLClassLoader.findClass(Unknown Source) ~[?:?] > at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] > at > org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentOnly(ComponentClassLoader.java:150) > ~[flink-dist-1.19.0.jar:1.19.0] > at > org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:113) > ~[flink-dist-1.19.0.jar:1.19.0] > at java.lang.ClassLoader.loadClass(Unknown Source) ~[?:?] > ... 43 more > {quote} > — > bug found: > * [The JAXB APIs completely removed from the JDK > 11]([https://stackoverflow.com/questions/43574426/how-to-resolve-java-lang-noclassdeffounderror-javax-xml-bind-jaxbexception)] > * when error occurs while connecting to oss, OSSErrorResponseHandler will be > accessed and then the exception will be thrown > — > how to fix: > * oss sdk has already "Remove the parser of JAXBResponseParser" with this > commit > [https://github.com/aliyun/aliyun-oss-java-sdk/commit/c7b567c40c3a31b9b08092f5a49b567eaf0f854b] > * therefore, just upgrade oss sdk version(the latest version is 3.17.4) > — > TODOs: > * I have fixed and tested locally(with 1.19) > * but I am confusing about the content of file > `flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE` > ** can it be generated automatically? -- This message was sent by Atlassian Jira (v8.20.10#820010)