showuon commented on a change in pull request #9947: URL: https://github.com/apache/kafka/pull/9947#discussion_r562312538
########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java ########## @@ -115,22 +115,29 @@ public void shouldCreateBaseDirectory() { @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(Paths.get(stateDir.getPath())); + assertPermissions(Paths.get(appDir.getPath())); + } + + private void assertPermissions(final Path path) { + 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> baseFilePermissions = Files.getPosixFilePermissions(path); Review comment: This path could be `baseFile` and also `appFile`. I think we'd better to rename it, ex: `filePermissions` or any better idea? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java ########## @@ -109,16 +109,27 @@ public StateDirectory(final StreamsConfig config, final Time time, final boolean 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(Paths.get(baseDir.getPath())); + configurePermissions(Paths.get(stateDir.getPath())); + } + } + + private void configurePermissions(final Path path) { + 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 { + final File file = path.toFile(); Review comment: We pass the `path` parameter which is converted from `File` instance (i.e. `Paths.get(stateDir.getPath()`), and then we converted it back to `File` instance in the method. It's redundant. Could we just pass the `File` instance into the method directly? ########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java ########## @@ -115,22 +115,29 @@ public void shouldCreateBaseDirectory() { @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(Paths.get(stateDir.getPath())); + assertPermissions(Paths.get(appDir.getPath())); + } + + private void assertPermissions(final Path path) { + 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> baseFilePermissions = Files.getPosixFilePermissions(path); + assertThat(expectedPermissions, equalTo(baseFilePermissions)); + } catch (final IOException e) { + fail("Should create correct files and set correct permissions"); + } + } else { + final File file = path.toFile(); Review comment: Same here, could we pass `File` instance into method directly? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org