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

Matthias Pohl commented on FLINK-26015:
---------------------------------------

I wasn't able to reproduce it with {{HadoopS3FileSystemBehaviorITCase}} using 
the following environment variables:
* {{IT_CASE_S3_BUCKET}}: {{foo}}
* {{IT_CASE_S3_ACCESS_KEY}}: {{minio}}
* {{IT_CASE_S3_SECRET_KEY}}: {{minio123}}

Additionally, we have to extend the configuration (in 
{{HadoopS3FileSystemBehaviorITCase.checkCredentialsAndSetup}}:
{code}
conf.setString("s3.endpoint", "http://localhost:9000";);
conf.setString("s3.path.style.access", "true");
{code}

A custom test still succeeded:
{code}
    @Test
    public void testAccessingADirectory() throws Exception {
        final Path dir = new Path(getBasePath(), randomName());
        fs.mkdirs(dir);

        FileStatus fileStatus = fs.getFileStatus(dir);
        assertNotNull(fileStatus);
        assertTrue(fileStatus.isDir());
    }
{code}

> FileSystemJobResultStore fails to access Minio
> ----------------------------------------------
>
>                 Key: FLINK-26015
>                 URL: https://issues.apache.org/jira/browse/FLINK-26015
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Runtime / Coordination
>    Affects Versions: 1.15.0
>            Reporter: Matthias Pohl
>            Assignee: Matthias Pohl
>            Priority: Critical
>         Attachments: failure.log.gz
>
>
> We're experiencing issues with accessing Minio-backed filesystems (probably 
> s3 object stores in general). The base directory appears to be not created.
> {code:java}
> 2022-02-08 13:13:31,682 ERROR 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint        [] - Fatal error 
> occurred in the cluster entrypoint.
> java.util.concurrent.CompletionException: 
> org.apache.flink.util.FlinkRuntimeException: Could not retrieve JobResults of 
> globally-terminated jobs from JobResultStore
>       at 
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
>  ~[?:1.8.0_322]
>       at 
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
>  [?:1.8.0_322]
>       at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1606)
>  [?:1.8.0_322]
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  [?:1.8.0_322]
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  [?:1.8.0_322]
>       at java.lang.Thread.run(Thread.java:750) [?:1.8.0_322]
> Caused by: org.apache.flink.util.FlinkRuntimeException: Could not retrieve 
> JobResults of globally-terminated jobs from JobResultStore
>       at 
> org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess.getDirtyJobResults(SessionDispatcherLeaderProcess.java:186)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess.supplyUnsynchronizedIfRunning(AbstractDispatcherLeaderProcess.java:198)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess.getDirtyJobResultsIfRunning(SessionDispatcherLeaderProcess.java:178)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
>  ~[?:1.8.0_322]
>       ... 3 more
> Caused by: java.io.FileNotFoundException: No such file or directory: 
> s3://store/myorg/myscope/3aa35e65-df86-4b16-8cc7-7c75af879317-test-job-name-aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa/ha/job-result-store/default
>       at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2344)
>  ~[?:?]
>       at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2226)
>  ~[?:?]
>       at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:2160) 
> ~[?:?]
>       at 
> org.apache.flink.fs.s3hadoop.common.HadoopFileSystem.getFileStatus(HadoopFileSystem.java:85)
>  ~[?:?]
>       at 
> org.apache.flink.core.fs.PluginFileSystemFactory$ClassLoaderFixingFileSystem.getFileStatus(PluginFileSystemFactory.java:105)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.highavailability.FileSystemJobResultStore.getDirtyResultsInternal(FileSystemJobResultStore.java:158)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.highavailability.AbstractThreadsafeJobResultStore.withReadLock(AbstractThreadsafeJobResultStore.java:118)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.highavailability.AbstractThreadsafeJobResultStore.getDirtyResults(AbstractThreadsafeJobResultStore.java:100)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess.getDirtyJobResults(SessionDispatcherLeaderProcess.java:184)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess.supplyUnsynchronizedIfRunning(AbstractDispatcherLeaderProcess.java:198)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess.getDirtyJobResultsIfRunning(SessionDispatcherLeaderProcess.java:178)
>  ~[flink-dist-flink-nightly.jar:flink-nightly]
>       at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
>  ~[?:1.8.0_322]
>       ... 3 more {code}



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

Reply via email to