[ 
https://issues.apache.org/jira/browse/FLINK-24763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17444082#comment-17444082
 ] 

Arvid Heise commented on FLINK-24763:
-------------------------------------

For me, it still not clear why the reader is closed at all. The reader should 
shutdown the fetcher and wait for termination and only after that should the 
classloader be closed.

If I look at the extracted log, I don't see any sign of shutdown before the 
failure

{noformat}
23:14:08,052 [flink-akka.actor.default-dispatcher-10] INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3) (4cdd9d2aec738c05342d7c2603a72cc8) switched from INITIALIZING to RUNNING.

23:14:08,068 [Source Data Fetcher for Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0] INFO  
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher [] - 
Starting split fetcher 0

23:14:08,276 [Source Data Fetcher for Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0] WARN  org.apache.hadoop.util.NativeCodeLoader                      [] 
- Unable to load native-hadoop library for your platform... using builtin-java 
classes where applicable

23:14:08,457 [Source Data Fetcher for Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0] INFO  org.apache.hadoop.io.compress.CodecPool                      [] 
- Got brand-new decompressor [.snappy]

23:14:08,463 [Source Data Fetcher for Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0] ERROR 
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager [] - 
Received uncaught exception.

23:14:08,474 [Source Data Fetcher for Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0] INFO  
org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher [] - Split 
fetcher 0 exited.

23:14:08,474 [Source: TableSourceScan(table=[[default_catalog, 
default_database, parquetLimitTable, limit=[5]]], fields=[x, y, a]) -> 
Limit(offset=[0], fetch=[5], global=[false]) (2/3)#0] INFO  
org.apache.flink.connector.base.source.reader.SourceReaderBase [] - Closing 
Source Reader.

23:14:08,478 [Source: TableSourceScan(table=[[default_catalog, 
default_database, parquetLimitTable, limit=[5]]], fields=[x, y, a]) -> 
Limit(offset=[0], fetch=[5], global=[false]) (2/3)#0] WARN  
org.apache.flink.runtime.taskmanager.Task                    [] - Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3)#0 (4cdd9d2aec738c05342d7c2603a72cc8) switched from RUNNING to FAILED with 
failure cause: java.lang.RuntimeException: One or more fetchers have 
encountered exception

23:14:08,479 [Source: TableSourceScan(table=[[default_catalog, 
default_database, parquetLimitTable, limit=[5]]], fields=[x, y, a]) -> 
Limit(offset=[0], fetch=[5], global=[false]) (2/3)#0] INFO  
org.apache.flink.runtime.taskmanager.Task                    [] - Freeing task 
resources for Source: TableSourceScan(table=[[default_catalog, 
default_database, parquetLimitTable, limit=[5]]], fields=[x, y, a]) -> 
Limit(offset=[0], fetch=[5], global=[false]) (2/3)#0 
(4cdd9d2aec738c05342d7c2603a72cc8).

23:14:08,484 [flink-akka.actor.default-dispatcher-10] INFO  
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - 
Un-registering task and sending final execution state FAILED to JobManager for 
task Source: TableSourceScan(table=[[default_catalog, default_database, 
parquetLimitTable, limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], 
fetch=[5], global=[false]) (2/3)#0 4cdd9d2aec738c05342d7c2603a72cc8.

23:14:08,498 [flink-akka.actor.default-dispatcher-7] INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
TableSourceScan(table=[[default_catalog, default_database, parquetLimitTable, 
limit=[5]]], fields=[x, y, a]) -> Limit(offset=[0], fetch=[5], global=[false]) 
(2/3) (4cdd9d2aec738c05342d7c2603a72cc8) switched from RUNNING to FAILED on 
736f6206-2099-4579-b8cd-cd737f09e9c9 @ localhost (dataPort=-1).
{noformat}

Unfortunately, I currently cannot execute this test in my IDE. Could you check 
when {{SafetyNetWrapperClassLoader#close}} is called on the respective failing 
task?


> ParquetFileSystemITCase.testLimitableBulkFormat failed on Azure
> ---------------------------------------------------------------
>
>                 Key: FLINK-24763
>                 URL: https://issues.apache.org/jira/browse/FLINK-24763
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / FileSystem, Formats (JSON, Avro, Parquet, 
> ORC, SequenceFile)
>    Affects Versions: 1.15.0
>            Reporter: Till Rohrmann
>            Assignee: Jingsong Lee
>            Priority: Critical
>              Labels: pull-request-available, test-stability
>             Fix For: 1.15.0
>
>
> The test {{ParquetFileSystemITCase.testLimitableBulkFormat}} fails with 
> {code}
> 2021-11-03T22:10:11.5106075Z Nov 03 22:10:11 [ERROR] 
> testLimitableBulkFormat[false]  Time elapsed: 9.177 s  <<< ERROR!
> 2021-11-03T22:10:11.5106643Z Nov 03 22:10:11 java.lang.RuntimeException: 
> Failed to fetch next result
> 2021-11-03T22:10:11.5107213Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:109)
> 2021-11-03T22:10:11.5111034Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultIterator.hasNext(CollectResultIterator.java:80)
> 2021-11-03T22:10:11.5112190Z Nov 03 22:10:11  at 
> org.apache.flink.table.planner.connectors.CollectDynamicSink$CloseableRowIteratorWrapper.hasNext(CollectDynamicSink.java:188)
> 2021-11-03T22:10:11.5112892Z Nov 03 22:10:11  at 
> java.util.Iterator.forEachRemaining(Iterator.java:115)
> 2021-11-03T22:10:11.5113393Z Nov 03 22:10:11  at 
> org.apache.flink.util.CollectionUtil.iteratorToList(CollectionUtil.java:109)
> 2021-11-03T22:10:11.5114157Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.ParquetFileSystemITCase.testLimitableBulkFormat(ParquetFileSystemITCase.java:128)
> 2021-11-03T22:10:11.5114951Z Nov 03 22:10:11  at 
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2021-11-03T22:10:11.5115568Z Nov 03 22:10:11  at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 2021-11-03T22:10:11.5116115Z Nov 03 22:10:11  at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 2021-11-03T22:10:11.5116591Z Nov 03 22:10:11  at 
> java.lang.reflect.Method.invoke(Method.java:498)
> 2021-11-03T22:10:11.5117088Z Nov 03 22:10:11  at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
> 2021-11-03T22:10:11.5117807Z Nov 03 22:10:11  at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> 2021-11-03T22:10:11.5118821Z Nov 03 22:10:11  at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
> 2021-11-03T22:10:11.5119417Z Nov 03 22:10:11  at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> 2021-11-03T22:10:11.5119944Z Nov 03 22:10:11  at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> 2021-11-03T22:10:11.5120427Z Nov 03 22:10:11  at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
> 2021-11-03T22:10:11.5120919Z Nov 03 22:10:11  at 
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
> 2021-11-03T22:10:11.5121571Z Nov 03 22:10:11  at 
> org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
> 2021-11-03T22:10:11.5122526Z Nov 03 22:10:11  at 
> org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
> 2021-11-03T22:10:11.5123245Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 2021-11-03T22:10:11.5123804Z Nov 03 22:10:11  at 
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
> 2021-11-03T22:10:11.5124314Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
> 2021-11-03T22:10:11.5124806Z Nov 03 22:10:11  at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
> 2021-11-03T22:10:11.5125313Z Nov 03 22:10:11  at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
> 2021-11-03T22:10:11.5125810Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
> 2021-11-03T22:10:11.5126281Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
> 2021-11-03T22:10:11.5126739Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
> 2021-11-03T22:10:11.5127349Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
> 2021-11-03T22:10:11.5128092Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
> 2021-11-03T22:10:11.5128984Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.run(ParentRunner.java:413)
> 2021-11-03T22:10:11.5129685Z Nov 03 22:10:11  at 
> org.junit.runners.Suite.runChild(Suite.java:128)
> 2021-11-03T22:10:11.5130330Z Nov 03 22:10:11  at 
> org.junit.runners.Suite.runChild(Suite.java:27)
> 2021-11-03T22:10:11.5130771Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
> 2021-11-03T22:10:11.5131222Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
> 2021-11-03T22:10:11.5131663Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
> 2021-11-03T22:10:11.5132139Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
> 2021-11-03T22:10:11.5132776Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
> 2021-11-03T22:10:11.5133441Z Nov 03 22:10:11  at 
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
> 2021-11-03T22:10:11.5134150Z Nov 03 22:10:11  at 
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
> 2021-11-03T22:10:11.5134816Z Nov 03 22:10:11  at 
> org.junit.rules.RunRules.evaluate(RunRules.java:20)
> 2021-11-03T22:10:11.5135741Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 2021-11-03T22:10:11.5136292Z Nov 03 22:10:11  at 
> org.junit.runners.ParentRunner.run(ParentRunner.java:413)
> 2021-11-03T22:10:11.5136717Z Nov 03 22:10:11  at 
> org.junit.runner.JUnitCore.run(JUnitCore.java:137)
> 2021-11-03T22:10:11.5137140Z Nov 03 22:10:11  at 
> org.junit.runner.JUnitCore.run(JUnitCore.java:115)
> 2021-11-03T22:10:11.5137603Z Nov 03 22:10:11  at 
> org.junit.vintage.engine.execution.RunnerExecutor.execute(RunnerExecutor.java:43)
> 2021-11-03T22:10:11.5138134Z Nov 03 22:10:11  at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
> 2021-11-03T22:10:11.5138766Z Nov 03 22:10:11  at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
> 2021-11-03T22:10:11.5139235Z Nov 03 22:10:11  at 
> java.util.Iterator.forEachRemaining(Iterator.java:116)
> 2021-11-03T22:10:11.5139733Z Nov 03 22:10:11  at 
> java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
> 2021-11-03T22:10:11.5140493Z Nov 03 22:10:11  at 
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
> 2021-11-03T22:10:11.5141265Z Nov 03 22:10:11  at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
> 2021-11-03T22:10:11.5141991Z Nov 03 22:10:11  at 
> java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
> 2021-11-03T22:10:11.5142892Z Nov 03 22:10:11  at 
> java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
> 2021-11-03T22:10:11.5143712Z Nov 03 22:10:11  at 
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
> 2021-11-03T22:10:11.5144655Z Nov 03 22:10:11  at 
> java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485)
> 2021-11-03T22:10:11.5145423Z Nov 03 22:10:11  at 
> org.junit.vintage.engine.VintageTestEngine.executeAllChildren(VintageTestEngine.java:82)
> 2021-11-03T22:10:11.5146236Z Nov 03 22:10:11  at 
> org.junit.vintage.engine.VintageTestEngine.execute(VintageTestEngine.java:73)
> 2021-11-03T22:10:11.5147106Z Nov 03 22:10:11  at 
> org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:220)
> 2021-11-03T22:10:11.5148061Z Nov 03 22:10:11  at 
> org.junit.platform.launcher.core.DefaultLauncher.lambda$execute$6(DefaultLauncher.java:188)
> 2021-11-03T22:10:11.5149081Z Nov 03 22:10:11  at 
> org.junit.platform.launcher.core.DefaultLauncher.withInterceptedStreams(DefaultLauncher.java:202)
> 2021-11-03T22:10:11.5149900Z Nov 03 22:10:11  at 
> org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:181)
> 2021-11-03T22:10:11.5150722Z Nov 03 22:10:11  at 
> org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:128)
> 2021-11-03T22:10:11.5151619Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invokeAllTests(JUnitPlatformProvider.java:150)
> 2021-11-03T22:10:11.5152790Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invoke(JUnitPlatformProvider.java:120)
> 2021-11-03T22:10:11.5153810Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
> 2021-11-03T22:10:11.5154754Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
> 2021-11-03T22:10:11.5155649Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
> 2021-11-03T22:10:11.5156235Z Nov 03 22:10:11  at 
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
> 2021-11-03T22:10:11.5156702Z Nov 03 22:10:11 Caused by: java.io.IOException: 
> Failed to fetch job execution result
> 2021-11-03T22:10:11.5157298Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:184)
> 2021-11-03T22:10:11.5157969Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.next(CollectResultFetcher.java:121)
> 2021-11-03T22:10:11.5158972Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:106)
> 2021-11-03T22:10:11.5160158Z Nov 03 22:10:11  ... 67 more
> 2021-11-03T22:10:11.5160863Z Nov 03 22:10:11 Caused by: 
> java.util.concurrent.ExecutionException: 
> org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> 2021-11-03T22:10:11.5161734Z Nov 03 22:10:11  at 
> java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
> 2021-11-03T22:10:11.5162618Z Nov 03 22:10:11  at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
> 2021-11-03T22:10:11.5163501Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:182)
> 2021-11-03T22:10:11.5164204Z Nov 03 22:10:11  ... 69 more
> 2021-11-03T22:10:11.5164752Z Nov 03 22:10:11 Caused by: 
> org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> 2021-11-03T22:10:11.5165533Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144)
> 2021-11-03T22:10:11.5166427Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$3(MiniClusterJobClient.java:137)
> 2021-11-03T22:10:11.5167353Z Nov 03 22:10:11  at 
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
> 2021-11-03T22:10:11.5168131Z Nov 03 22:10:11  at 
> java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:628)
> 2021-11-03T22:10:11.5169012Z Nov 03 22:10:11  at 
> java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1996)
> 2021-11-03T22:10:11.5169614Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.minicluster.MiniClusterJobClient.getJobExecutionResult(MiniClusterJobClient.java:134)
> 2021-11-03T22:10:11.5170488Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:181)
> 2021-11-03T22:10:11.5170942Z Nov 03 22:10:11  ... 69 more
> 2021-11-03T22:10:11.5171362Z Nov 03 22:10:11 Caused by: 
> org.apache.flink.runtime.JobException: Recovery is suppressed by 
> NoRestartBackoffTimeStrategy
> 2021-11-03T22:10:11.5171990Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138)
> 2021-11-03T22:10:11.5172937Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82)
> 2021-11-03T22:10:11.5173707Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:228)
> 2021-11-03T22:10:11.5174364Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:218)
> 2021-11-03T22:10:11.5174999Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:209)
> 2021-11-03T22:10:11.5175621Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:681)
> 2021-11-03T22:10:11.5176374Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79)
> 2021-11-03T22:10:11.5177185Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:445)
> 2021-11-03T22:10:11.5177900Z Nov 03 22:10:11  at 
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2021-11-03T22:10:11.5178571Z Nov 03 22:10:11  at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 2021-11-03T22:10:11.5179526Z Nov 03 22:10:11  at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 2021-11-03T22:10:11.5180271Z Nov 03 22:10:11  at 
> java.lang.reflect.Method.invoke(Method.java:498)
> 2021-11-03T22:10:11.5181030Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:316)
> 2021-11-03T22:10:11.5182018Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
> 2021-11-03T22:10:11.5183204Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:314)
> 2021-11-03T22:10:11.5183790Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217)
> 2021-11-03T22:10:11.5184385Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78)
> 2021-11-03T22:10:11.5184939Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163)
> 2021-11-03T22:10:11.5185462Z Nov 03 22:10:11  at 
> akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)
> 2021-11-03T22:10:11.5185923Z Nov 03 22:10:11  at 
> akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)
> 2021-11-03T22:10:11.5186504Z Nov 03 22:10:11  at 
> scala.PartialFunction.applyOrElse(PartialFunction.scala:123)
> 2021-11-03T22:10:11.5186966Z Nov 03 22:10:11  at 
> scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)
> 2021-11-03T22:10:11.5187419Z Nov 03 22:10:11  at 
> akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)
> 2021-11-03T22:10:11.5187885Z Nov 03 22:10:11  at 
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> 2021-11-03T22:10:11.5188346Z Nov 03 22:10:11  at 
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> 2021-11-03T22:10:11.5188890Z Nov 03 22:10:11  at 
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> 2021-11-03T22:10:11.5189566Z Nov 03 22:10:11  at 
> akka.actor.Actor.aroundReceive(Actor.scala:537)
> 2021-11-03T22:10:11.5190267Z Nov 03 22:10:11  at 
> akka.actor.Actor.aroundReceive$(Actor.scala:535)
> 2021-11-03T22:10:11.5190700Z Nov 03 22:10:11  at 
> akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)
> 2021-11-03T22:10:11.5191170Z Nov 03 22:10:11  at 
> akka.actor.ActorCell.receiveMessage(ActorCell.scala:580)
> 2021-11-03T22:10:11.5191586Z Nov 03 22:10:11  at 
> akka.actor.ActorCell.invoke(ActorCell.scala:548)
> 2021-11-03T22:10:11.5192019Z Nov 03 22:10:11  at 
> akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)
> 2021-11-03T22:10:11.5192544Z Nov 03 22:10:11  at 
> akka.dispatch.Mailbox.run(Mailbox.scala:231)
> 2021-11-03T22:10:11.5192937Z Nov 03 22:10:11  at 
> akka.dispatch.Mailbox.exec(Mailbox.scala:243)
> 2021-11-03T22:10:11.5193584Z Nov 03 22:10:11  at 
> java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
> 2021-11-03T22:10:11.5194359Z Nov 03 22:10:11  at 
> java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
> 2021-11-03T22:10:11.5195082Z Nov 03 22:10:11  at 
> java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
> 2021-11-03T22:10:11.5195807Z Nov 03 22:10:11  at 
> java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
> 2021-11-03T22:10:11.5196524Z Nov 03 22:10:11 Caused by: 
> java.lang.RuntimeException: One or more fetchers have encountered exception
> 2021-11-03T22:10:11.5197410Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:225)
> 2021-11-03T22:10:11.5198368Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:169)
> 2021-11-03T22:10:11.5199370Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:130)
> 2021-11-03T22:10:11.5200292Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:350)
> 2021-11-03T22:10:11.5201255Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
> 2021-11-03T22:10:11.5202193Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
> 2021-11-03T22:10:11.5203230Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:474)
> 2021-11-03T22:10:11.5203819Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203)
> 2021-11-03T22:10:11.5204599Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:788)
> 2021-11-03T22:10:11.5205415Z Nov 03 22:10:11  at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:737)
> 2021-11-03T22:10:11.5206119Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:947)
> 2021-11-03T22:10:11.5206635Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:926)
> 2021-11-03T22:10:11.5207306Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:740)
> 2021-11-03T22:10:11.5207928Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)
> 2021-11-03T22:10:11.5208474Z Nov 03 22:10:11  at 
> java.lang.Thread.run(Thread.java:748)
> 2021-11-03T22:10:11.5209343Z Nov 03 22:10:11 Caused by: 
> java.lang.RuntimeException: SplitFetcher thread 0 received unexpected 
> exception while polling the records
> 2021-11-03T22:10:11.5209976Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:150)
> 2021-11-03T22:10:11.5210570Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:105)
> 2021-11-03T22:10:11.5211092Z Nov 03 22:10:11  at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 2021-11-03T22:10:11.5211564Z Nov 03 22:10:11  at 
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 2021-11-03T22:10:11.5212180Z Nov 03 22:10:11  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 2021-11-03T22:10:11.5212969Z Nov 03 22:10:11  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 2021-11-03T22:10:11.5213361Z Nov 03 22:10:11  ... 1 more
> 2021-11-03T22:10:11.5215786Z Nov 03 22:10:11 Caused by: 
> java.lang.IllegalStateException: Trying to access closed classloader. Please 
> check if you store classloaders directly or indirectly in static fields. If 
> the stacktrace suggests that the leak occurs in a third party library and 
> cannot be fixed immediately, you can disable this check with the 
> configuration 'classloader.check-leaked-classloader'.
> 2021-11-03T22:10:11.5217523Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164)
> 2021-11-03T22:10:11.5218577Z Nov 03 22:10:11  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.getResource(FlinkUserCodeClassLoaders.java:183)
> 2021-11-03T22:10:11.5219513Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2780)
> 2021-11-03T22:10:11.5220068Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3036)
> 2021-11-03T22:10:11.5220721Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2995)
> 2021-11-03T22:10:11.5221505Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2968)
> 2021-11-03T22:10:11.5222138Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2848)
> 2021-11-03T22:10:11.5222733Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.get(Configuration.java:1200)
> 2021-11-03T22:10:11.5223230Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1254)
> 2021-11-03T22:10:11.5223707Z Nov 03 22:10:11  at 
> org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1479)
> 2021-11-03T22:10:11.5224231Z Nov 03 22:10:11  at 
> org.apache.parquet.hadoop.codec.SnappyCodec.createInputStream(SnappyCodec.java:75)
> 2021-11-03T22:10:11.5224772Z Nov 03 22:10:11  at 
> org.apache.parquet.hadoop.CodecFactory$HeapBytesDecompressor.decompress(CodecFactory.java:109)
> 2021-11-03T22:10:11.5225418Z Nov 03 22:10:11  at 
> org.apache.parquet.hadoop.ColumnChunkPageReadStore$ColumnChunkPageReader.readDictionaryPage(ColumnChunkPageReadStore.java:196)
> 2021-11-03T22:10:11.5226286Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader.<init>(AbstractColumnReader.java:110)
> 2021-11-03T22:10:11.5226876Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.vector.reader.IntColumnReader.<init>(IntColumnReader.java:33)
> 2021-11-03T22:10:11.5227492Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil.createColumnReader(ParquetSplitReaderUtil.java:280)
> 2021-11-03T22:10:11.5228185Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.readNextRowGroup(ParquetVectorizedInputFormat.java:412)
> 2021-11-03T22:10:11.5228961Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.nextBatch(ParquetVectorizedInputFormat.java:381)
> 2021-11-03T22:10:11.5229660Z Nov 03 22:10:11  at 
> org.apache.flink.formats.parquet.ParquetVectorizedInputFormat$ParquetReader.readBatch(ParquetVectorizedInputFormat.java:358)
> 2021-11-03T22:10:11.5230333Z Nov 03 22:10:11  at 
> org.apache.flink.table.filesystem.LimitableBulkFormat$LimitableReader.readBatch(LimitableBulkFormat.java:108)
> 2021-11-03T22:10:11.5230939Z Nov 03 22:10:11  at 
> org.apache.flink.connector.file.src.impl.FileSourceSplitReader.fetch(FileSourceSplitReader.java:67)
> 2021-11-03T22:10:11.5231515Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:58)
> 2021-11-03T22:10:11.5232095Z Nov 03 22:10:11  at 
> org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:142)
> 2021-11-03T22:10:11.5232614Z Nov 03 22:10:11  ... 6 more
> 2021-11-03T22:10:11.5232979Z Nov 03 22:10:11 
> 2021-11-03T22:10:11.5234489Z Exception in thread "Thread-11" 
> java.lang.IllegalStateException: Trying to access closed classloader. Please 
> check if you store classloaders directly or indirectly in static fields. If 
> the stacktrace suggests that the leak occurs in a third party library and 
> cannot be fixed immediately, you can disable this check with the 
> configuration 'classloader.check-leaked-classloader'.
> 2021-11-03T22:10:11.5235610Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164)
> 2021-11-03T22:10:11.5236345Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.getResource(FlinkUserCodeClassLoaders.java:183)
> 2021-11-03T22:10:11.5236944Z  at 
> org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2780)
> 2021-11-03T22:10:11.5237383Z  at 
> org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3036)
> 2021-11-03T22:10:11.5237843Z  at 
> org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2995)
> 2021-11-03T22:10:11.5238296Z  at 
> org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2968)
> 2021-11-03T22:10:11.5238798Z  at 
> org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2848)
> 2021-11-03T22:10:11.5239225Z  at 
> org.apache.hadoop.conf.Configuration.get(Configuration.java:1200)
> 2021-11-03T22:10:11.5239647Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1812)
> 2021-11-03T22:10:11.5240102Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1789)
> 2021-11-03T22:10:11.5240583Z  at 
> org.apache.hadoop.util.ShutdownHookManager.getShutdownTimeout(ShutdownHookManager.java:183)
> 2021-11-03T22:10:11.5241070Z  at 
> org.apache.hadoop.util.ShutdownHookManager.shutdownExecutor(ShutdownHookManager.java:145)
> 2021-11-03T22:10:11.5241561Z  at 
> org.apache.hadoop.util.ShutdownHookManager.access$300(ShutdownHookManager.java:65)
> 2021-11-03T22:10:11.5242032Z  at 
> org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:102)
> 2021-11-03T22:10:12.8086663Z Nov 03 22:10:12 [INFO] Tests run: 3, Failures: 
> 0, Errors: 0, Skipped: 0, Time elapsed: 16.576 s - in 
> org.apache.flink.formats.parquet.ParquetFileCompactionITCase
> 2021-11-03T22:10:12.8135630Z Exception in thread "Thread-10" 
> java.lang.IllegalStateException: Trying to access closed classloader. Please 
> check if you store classloaders directly or indirectly in static fields. If 
> the stacktrace suggests that the leak occurs in a third party library and 
> cannot be fixed immediately, you can disable this check with the 
> configuration 'classloader.check-leaked-classloader'.
> 2021-11-03T22:10:12.8137964Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164)
> 2021-11-03T22:10:12.8139304Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.getResource(FlinkUserCodeClassLoaders.java:183)
> 2021-11-03T22:10:12.8140308Z  at 
> org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2780)
> 2021-11-03T22:10:12.8141050Z  at 
> org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3036)
> 2021-11-03T22:10:12.8141824Z  at 
> org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2995)
> 2021-11-03T22:10:12.8142673Z  at 
> org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2968)
> 2021-11-03T22:10:12.8143402Z  at 
> org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2848)
> 2021-11-03T22:10:12.8144120Z  at 
> org.apache.hadoop.conf.Configuration.get(Configuration.java:1200)
> 2021-11-03T22:10:12.8144880Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1812)
> 2021-11-03T22:10:12.8145632Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1789)
> 2021-11-03T22:10:12.8146463Z  at 
> org.apache.hadoop.util.ShutdownHookManager.getShutdownTimeout(ShutdownHookManager.java:183)
> 2021-11-03T22:10:12.8147479Z  at 
> org.apache.hadoop.util.ShutdownHookManager.shutdownExecutor(ShutdownHookManager.java:145)
> 2021-11-03T22:10:12.8148320Z  at 
> org.apache.hadoop.util.ShutdownHookManager.access$300(ShutdownHookManager.java:65)
> 2021-11-03T22:10:12.8149186Z  at 
> org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:102)
> 2021-11-03T22:10:13.1599258Z Nov 03 22:10:13 [INFO] Running 
> org.apache.flink.formats.parquet.ParquetFsStreamingSinkITCase
> 2021-11-03T22:10:28.5117719Z Nov 03 22:10:28 [INFO] Tests run: 3, Failures: 
> 0, Errors: 0, Skipped: 0, Time elapsed: 15.346 s - in 
> org.apache.flink.formats.parquet.ParquetFsStreamingSinkITCase
> 2021-11-03T22:10:28.5155684Z Exception in thread "Thread-11" 
> java.lang.IllegalStateException: Trying to access closed classloader. Please 
> check if you store classloaders directly or indirectly in static fields. If 
> the stacktrace suggests that the leak occurs in a third party library and 
> cannot be fixed immediately, you can disable this check with the 
> configuration 'classloader.check-leaked-classloader'.
> 2021-11-03T22:10:28.5157699Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.ensureInner(FlinkUserCodeClassLoaders.java:164)
> 2021-11-03T22:10:28.5158964Z  at 
> org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders$SafetyNetWrapperClassLoader.getResource(FlinkUserCodeClassLoaders.java:183)
> 2021-11-03T22:10:28.5159902Z  at 
> org.apache.hadoop.conf.Configuration.getResource(Configuration.java:2780)
> 2021-11-03T22:10:28.5160644Z  at 
> org.apache.hadoop.conf.Configuration.getStreamReader(Configuration.java:3036)
> 2021-11-03T22:10:28.5161387Z  at 
> org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2995)
> 2021-11-03T22:10:28.5162100Z  at 
> org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2968)
> 2021-11-03T22:10:28.5162937Z  at 
> org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2848)
> 2021-11-03T22:10:28.5163616Z  at 
> org.apache.hadoop.conf.Configuration.get(Configuration.java:1200)
> 2021-11-03T22:10:28.5164320Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1812)
> 2021-11-03T22:10:28.5165054Z  at 
> org.apache.hadoop.conf.Configuration.getTimeDuration(Configuration.java:1789)
> 2021-11-03T22:10:28.5165827Z  at 
> org.apache.hadoop.util.ShutdownHookManager.getShutdownTimeout(ShutdownHookManager.java:183)
> 2021-11-03T22:10:28.5166962Z  at 
> org.apache.hadoop.util.ShutdownHookManager.shutdownExecutor(ShutdownHookManager.java:145)
> 2021-11-03T22:10:28.5167777Z  at 
> org.apache.hadoop.util.ShutdownHookManager.access$300(ShutdownHookManager.java:65)
> 2021-11-03T22:10:28.5168523Z  at 
> org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:102)
> {code}
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=25896&view=logs&j=ba53eb01-1462-56a3-8e98-0dd97fbcaab5&t=2e426bf0-b717-56bb-ab62-d63086457354&l=13356



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to