This is an automated email from the ASF dual-hosted git repository.

ableegoldman pushed a commit to branch 2.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.7 by this push:
     new dbc039d  KAFKA-12190: Fix setting of file permissions on non-POSIX 
filesystems (#9947)
dbc039d is described below

commit dbc039d650e9cff2f99530e7cb88643c0969edba
Author: Andy Wilkinson <[email protected]>
AuthorDate: Mon Jan 25 19:33:05 2021 +0000

    KAFKA-12190: Fix setting of file permissions on non-POSIX filesystems 
(#9947)
    
    Previously, StateDirectory used PosixFilePermissions to configure its 
directories' permissions which fails on Windows as its file system is not 
POSIX-compliant. This PR updates StateDirectory to fall back to the File API on 
non-POSIX-compliant file systems.
    
    Reviewers: Luke Chen <[email protected]>, Anna 
Sophie Blee-Goldman <[email protected]>
---
 .../processor/internals/StateDirectory.java        | 24 +++++++++----
 .../processor/internals/StateDirectoryTest.java    | 40 +++++++++++++---------
 2 files changed, 40 insertions(+), 24 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
index b98c3cb..e388b6a 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java
@@ -31,7 +31,6 @@ import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
 import java.nio.file.NoSuchFileException;
-import java.nio.file.Paths;
 import java.nio.file.Path;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
@@ -109,16 +108,27 @@ public class StateDirectory {
                 log.warn("Using /tmp directory in the state.dir property can 
cause failures with writing the checkpoint file" +
                     " due to the fact that this directory can be cleared by 
the OS");
             }
-
             // change the dir permission to "rwxr-x---" to avoid world readable
-            final Path basePath = Paths.get(baseDir.getPath());
-            final Path statePath = Paths.get(stateDir.getPath());
+            configurePermissions(baseDir);
+            configurePermissions(stateDir);
+        }
+    }
+    
+    private void configurePermissions(final File file) {
+        final Path path = file.toPath();
+        if 
(path.getFileSystem().supportedFileAttributeViews().contains("posix")) {
             final Set<PosixFilePermission> perms = 
PosixFilePermissions.fromString("rwxr-x---");
             try {
-                Files.setPosixFilePermissions(basePath, perms);
-                Files.setPosixFilePermissions(statePath, perms);
+                Files.setPosixFilePermissions(path, perms);
             } catch (final IOException e) {
-                log.error("Error changing permissions for the state or base 
directory {} ", stateDir.getPath(), e);
+                log.error("Error changing permissions for the directory {} ", 
path, e);
+            }
+        } else {
+            boolean set = file.setReadable(true, true);
+            set &= file.setWritable(true, true);
+            set &= file.setExecutable(true, true);
+            if (!set) {
+                log.error("Failed to change permissions for the directory {}", 
file);
             }
         }
     }
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java
index 5048962..f5f04dc 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java
@@ -35,7 +35,6 @@ import java.nio.channels.FileChannel;
 import java.nio.channels.OverlappingFileLockException;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.PosixFilePermission;
 import java.time.Duration;
@@ -115,22 +114,29 @@ public class StateDirectoryTest {
 
     @Test
     public void shouldHaveSecurePermissions() {
-        final Set<PosixFilePermission> expectedPermissions = EnumSet.of(
-            PosixFilePermission.OWNER_EXECUTE,
-            PosixFilePermission.GROUP_READ,
-            PosixFilePermission.OWNER_WRITE,
-            PosixFilePermission.GROUP_EXECUTE,
-            PosixFilePermission.OWNER_READ);
-
-        final Path statePath = Paths.get(stateDir.getPath());
-        final Path basePath = Paths.get(appDir.getPath());
-        try {
-            final Set<PosixFilePermission> baseFilePermissions = 
Files.getPosixFilePermissions(statePath);
-            final Set<PosixFilePermission> appFilePermissions = 
Files.getPosixFilePermissions(basePath);
-            assertThat(expectedPermissions, equalTo(baseFilePermissions));
-            assertThat(expectedPermissions, equalTo(appFilePermissions));
-        } catch (final IOException e) {
-            fail("Should create correct files and set correct permissions");
+        assertPermissions(stateDir);
+        assertPermissions(appDir);
+    }
+    
+    private void assertPermissions(final File file) {
+        final Path path = file.toPath();
+        if 
(path.getFileSystem().supportedFileAttributeViews().contains("posix")) {
+            final Set<PosixFilePermission> expectedPermissions = EnumSet.of(
+                    PosixFilePermission.OWNER_EXECUTE,
+                    PosixFilePermission.GROUP_READ,
+                    PosixFilePermission.OWNER_WRITE,
+                    PosixFilePermission.GROUP_EXECUTE,
+                    PosixFilePermission.OWNER_READ);
+            try {
+                final Set<PosixFilePermission> filePermissions = 
Files.getPosixFilePermissions(path);
+                assertThat(expectedPermissions, equalTo(filePermissions));
+            } catch (final IOException e) {
+                fail("Should create correct files and set correct 
permissions");
+            }
+        } else {
+            assertThat(file.canRead(), is(true));
+            assertThat(file.canWrite(), is(true));
+            assertThat(file.canExecute(), is(true));
         }
     }
 

Reply via email to