zentol commented on a change in pull request #18979:
URL: https://github.com/apache/flink/pull/18979#discussion_r822528770



##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);
+        }
+
+        return true;
+    }
+
+    private void deleteRecursively(Path path) throws IOException {
+        final FileStatus[] containingFiles = listStatus(path);
+        if (containingFiles == null) {
+            LOG.warn(
+                    "No files could be retrieved even though the path was 
marked as existing. "
+                            + "This is an indication for a bug in the 
underlying FileSystem "
+                            + "implementation and should be reported. It won't 
affect the "
+                            + "processing of this run, though.");
+            return;
+        }
+
+        IOException exception = null;
+        for (FileStatus fileStatus : containingFiles) {
+            final Path childPath = fileStatus.getPath();
+
+            try {
+                if (fileStatus.isDir()) {
+                    deleteRecursively(childPath);
+                } else {
+                    deleteObject(childPath);
+                }
+            } catch (IOException e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+        }
+
+        if (exception != null) {
+            throw exception;
+        }
+
+        if (containingFiles.length == 0) {
+            // Presto doesn't hold placeholders for directories itself; 
therefore, we don't need to
+            // call deleteObject on the directory itself if there were objects 
with the prefix being
+            // deleted in the initial loop above. This saves us from doing 
some existence checks on
+            // an not-existing object (i.e. the now empty directory)
+            deleteObject(path);
+        }
+    }
+
+    /**
+     * Deletes the object referenced by the passed {@code path}.
+     *
+     * @param path The path referring to the object that shall be deleted.
+     * @throws IOException if an error occurred while deleting the file other 
than the {@code path}
+     *     referring to a non-empty directory.
+     */
+    private void deleteObject(Path path) throws IOException {
+        boolean success = true;
+        IOException actualException = null;
+        try {
+            // this call will fail in case of the path representing an empty 
directory but the
+            // subsequent exist call will return false which leads to the 
desired behavior
+            // because for Presto, there is actually no object that needs to 
be deleted when
+            // referring to an empty directory
+            success = super.delete(path, false);

Review comment:
       If mkdirs is a no-op, and the directory marker is automatically deleted 
if the directory contents are removed, then why are we even trying to delete an 
empty directory in the first place?
   Sounds like we could just skip this step. Either the directory doesn't 
actually exist, or we cleaned up all contents implicitly deleting it.

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */

Review comment:
       ```suggestion
   ```

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);

Review comment:
       Why is this not just calling super.delete()?

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);
+        }
+
+        return true;
+    }
+
+    private void deleteRecursively(Path path) throws IOException {
+        final FileStatus[] containingFiles = listStatus(path);
+        if (containingFiles == null) {
+            LOG.warn(
+                    "No files could be retrieved even though the path was 
marked as existing. "
+                            + "This is an indication for a bug in the 
underlying FileSystem "
+                            + "implementation and should be reported. It won't 
affect the "
+                            + "processing of this run, though.");
+            return;
+        }
+
+        IOException exception = null;
+        for (FileStatus fileStatus : containingFiles) {
+            final Path childPath = fileStatus.getPath();
+
+            try {
+                if (fileStatus.isDir()) {
+                    deleteRecursively(childPath);
+                } else {
+                    deleteObject(childPath);
+                }
+            } catch (IOException e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+        }
+
+        if (exception != null) {
+            throw exception;
+        }
+
+        if (containingFiles.length == 0) {
+            // Presto doesn't hold placeholders for directories itself; 
therefore, we don't need to
+            // call deleteObject on the directory itself if there were objects 
with the prefix being
+            // deleted in the initial loop above. This saves us from doing 
some existence checks on
+            // an not-existing object (i.e. the now empty directory)
+            deleteObject(path);
+        }

Review comment:
       This looks like an early exit condition; why is it done at the end?

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);
+        }
+
+        return true;
+    }
+
+    private void deleteRecursively(Path path) throws IOException {
+        final FileStatus[] containingFiles = listStatus(path);
+        if (containingFiles == null) {
+            LOG.warn(
+                    "No files could be retrieved even though the path was 
marked as existing. "
+                            + "This is an indication for a bug in the 
underlying FileSystem "
+                            + "implementation and should be reported. It won't 
affect the "
+                            + "processing of this run, though.");
+            return;
+        }
+
+        IOException exception = null;
+        for (FileStatus fileStatus : containingFiles) {
+            final Path childPath = fileStatus.getPath();
+
+            try {
+                if (fileStatus.isDir()) {
+                    deleteRecursively(childPath);
+                } else {
+                    deleteObject(childPath);
+                }
+            } catch (IOException e) {
+                exception = ExceptionUtils.firstOrSuppressed(e, exception);
+            }
+        }
+
+        if (exception != null) {
+            throw exception;
+        }
+
+        if (containingFiles.length == 0) {
+            // Presto doesn't hold placeholders for directories itself; 
therefore, we don't need to
+            // call deleteObject on the directory itself if there were objects 
with the prefix being
+            // deleted in the initial loop above. This saves us from doing 
some existence checks on
+            // an not-existing object (i.e. the now empty directory)
+            deleteObject(path);
+        }

Review comment:
       I don't get why it's necessary in the first place; see below.

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);
+        }
+
+        return true;
+    }
+
+    private void deleteRecursively(Path path) throws IOException {
+        final FileStatus[] containingFiles = listStatus(path);
+        if (containingFiles == null) {
+            LOG.warn(
+                    "No files could be retrieved even though the path was 
marked as existing. "
+                            + "This is an indication for a bug in the 
underlying FileSystem "
+                            + "implementation and should be reported. It won't 
affect the "
+                            + "processing of this run, though.");

Review comment:
       It also doesn't make sense to just return in this case; we have no idea 
what the actual external state is because we already assume the underlying FS 
to be broken.

##########
File path: 
flink-core/src/test/java/org/apache/flink/core/fs/FileSystemBehaviorTestSuite.java
##########
@@ -97,6 +99,92 @@ public void testHomeAndWorkDir() {
         assertEquals(fs.getUri().getScheme(), 
fs.getWorkingDirectory().toUri().getScheme());
         assertEquals(fs.getUri().getScheme(), 
fs.getHomeDirectory().toUri().getScheme());
     }
+    // --- exists
+
+    @Test
+    public void testFileExists() throws IOException {
+        final Path filePath = createRandomFileInDirectory(basePath);
+        assertTrue(fs.exists(filePath));
+    }
+
+    @Test
+    public void testFileDoesNotExist() throws IOException {
+        assertFalse(fs.exists(new Path(basePath, randomName())));
+    }
+
+    // --- delete
+
+    @Test
+    public void testExistingFileDeletion() throws IOException {
+        testSuccessfulDeletion(createRandomFileInDirectory(basePath), false);
+    }
+
+    @Test
+    public void testExistingFileRecursiveDeletion() throws IOException {
+        testSuccessfulDeletion(createRandomFileInDirectory(basePath), true);
+    }
+
+    @Test
+    public void testNotExistingFileDeletion() throws IOException {
+        testSuccessfulDeletion(new Path(basePath, randomName()), false);
+    }
+
+    @Test
+    public void testNotExistingFileRecursiveDeletion() throws IOException {
+        testSuccessfulDeletion(new Path(basePath, randomName()), true);
+    }
+
+    @Test
+    public void testExistingEmptyDirectoryDeletion() throws IOException {
+        final Path path = new Path(basePath, "dir-" + randomName());
+        fs.mkdirs(path);
+        testSuccessfulDeletion(path, false);
+    }
+
+    @Test
+    public void testExistingEmptyDirectoryRecursiveDeletion() throws 
IOException {
+        final Path path = new Path(basePath, "dir-" + randomName());

Review comment:
       Is the `dir-` prefix necessary?

##########
File path: 
flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/FlinkS3PrestoFileSystem.java
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.fs.s3presto;
+
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.fs.s3.common.FlinkS3FileSystem;
+import org.apache.flink.fs.s3.common.writer.S3AccessHelper;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.nio.file.DirectoryNotEmptyException;
+import java.util.Optional;
+
+/**
+ * {@code FlinkS3PrestoFileSystem} provides custom recursive deletion 
functionality to work around a
+ * bug in the internally used Presto file system.
+ *
+ * <p>https://github.com/prestodb/presto/issues/17416
+ */
+public class FlinkS3PrestoFileSystem extends FlinkS3FileSystem {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(FlinkS3PrestoFileSystem.class);
+
+    /**
+     * Creates a {@code FlinkS3PrestoFileSystem} based on the given Hadoop S3 
file system. The given
+     * Hadoop file system object is expected to be initialized already.
+     *
+     * <p>This constructor additionally configures the entropy injection for 
the file system.
+     *
+     * @param hadoopS3FileSystem The Hadoop FileSystem that will be used under 
the hood.
+     * @param entropyInjectionKey The substring that will be replaced by 
entropy or removed.
+     * @param entropyLength The number of random alphanumeric characters to 
inject as entropy.
+     */
+    public FlinkS3PrestoFileSystem(
+            FileSystem hadoopS3FileSystem,
+            String localTmpDirectory,
+            @Nullable String entropyInjectionKey,
+            int entropyLength,
+            @Nullable S3AccessHelper s3UploadHelper,
+            long s3uploadPartSize,
+            int maxConcurrentUploadsPerStream) {
+        super(
+                hadoopS3FileSystem,
+                localTmpDirectory,
+                entropyInjectionKey,
+                entropyLength,
+                s3UploadHelper,
+                s3uploadPartSize,
+                maxConcurrentUploadsPerStream);
+    }
+
+    @Override
+    public boolean delete(Path path, boolean recursive) throws IOException {
+        if (recursive) {
+            deleteRecursively(path);
+        } else {
+            if (isDirectoryWithContent(path)) {
+                throw new DirectoryNotEmptyException(path.getPath());
+            }
+
+            deleteObject(path);
+        }
+
+        return true;
+    }
+
+    private void deleteRecursively(Path path) throws IOException {
+        final FileStatus[] containingFiles = listStatus(path);
+        if (containingFiles == null) {
+            LOG.warn(
+                    "No files could be retrieved even though the path was 
marked as existing. "
+                            + "This is an indication for a bug in the 
underlying FileSystem "
+                            + "implementation and should be reported. It won't 
affect the "
+                            + "processing of this run, though.");

Review comment:
       If it is not a problem, why log it?
   
   This message is also not comprehensible for users; it doesn't mention 
listing files or even directories.




-- 
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: issues-unsubscr...@flink.apache.org

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


Reply via email to