Skip to content

KAFKA-12190: Fix setting of file permissions on non-POSIX filesystems#9947

Merged
ableegoldman merged 3 commits intoapache:trunkfrom
wilkinsona:KAFKA-12190
Jan 25, 2021
Merged

KAFKA-12190: Fix setting of file permissions on non-POSIX filesystems#9947
ableegoldman merged 3 commits intoapache:trunkfrom
wilkinsona:KAFKA-12190

Conversation

@wilkinsona
Copy link
Copy Markdown
Contributor

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. The File API doesn't allow as much control over the permissions so they're as close as the API permits.

The unit tests have been updated to also verify the behaviour on non-POSIX-compliant file systems.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the fix! I had a few questions about how to set the permissions for non-users, but I may be out of my depth when it comes to non-posix permissions so let me know if I'm just not on the same page here.

Copy link
Copy Markdown
Member

@showuon showuon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the fix. Overall looks good. Left some comments. Thanks.

@wilkinsona
Copy link
Copy Markdown
Contributor Author

Thanks for the reviews. I've addressed the comments other than possibly passing the File into the configurePermissions and assertPermissions methods. Let me know which way you want to go there.

@wilkinsona
Copy link
Copy Markdown
Contributor Author

wilkinsona commented Jan 25, 2021

I think all the review comments have now been addressed. Please let me know if you'd like the commits to be squashed into one.

Copy link
Copy Markdown
Member

@showuon showuon left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks a lot.

@ableegoldman
Copy link
Copy Markdown
Member

Don't worry about squashing all the commits yourself, we'll do that automatically when merging it.

Three test failures in the build, all unrelated:

org.apache.kafka.clients.consumer.KafkaConsumerTest.testCloseWithTimeUnit()
org.apache.kafka.clients.consumer.internals.FetcherTest.testEarlierOffsetResetArrivesLate()
org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest.testReplication()

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks for the patch

@ableegoldman ableegoldman merged commit e1a4dcc into apache:trunk Jan 25, 2021
ableegoldman pushed a commit that referenced this pull request Jan 25, 2021
…#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 <43372967+showuon@users.noreply.github.com>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
@ableegoldman
Copy link
Copy Markdown
Member

Merged to trunk and cherrypicked to 2.7

@ableegoldman
Copy link
Copy Markdown
Member

I tried to cherrypick this back to the 2.6 branch but there was a merge conflict. I would be happy to review & merge another PR if you'd like to open one against the 2.6 branch, but realistically I think it's unlikely that we'll have a 2.6.2 release just given the lack of x.x.2 releases in recent history.

We can also just keep an eye out for a proposed 2.6.2 release and then work on another patch for the 2.6 branch if that does happen

@wilkinsona
Copy link
Copy Markdown
Contributor Author

Thanks for the merge.

Given that the regression in 2.6.1 makes Streams unusable for developers working on Windows, I really think that a 2.6.2 release is needed here. To that end, I've opened a new pull request based on the 2.6 branch.

@garyrussell
Copy link
Copy Markdown

As I commented on the JIRA issue I think it is important to get a 2.6.2 release out sooner rather than later.

@ableegoldman
Copy link
Copy Markdown
Member

@garyrussell I just merged the 2.6 PR so we'll be ready to go if/when the 2.6.2 release is started. To be honest I'm not sure how likely it is for a 2.6.2 release to be rushed out. It's a lot of work to roll out an AK release, even a bugfix. It might be difficult to convince someone to take all that on as a release manager in order to fix a regression that impacts a relatively small subset of users (this is not my personal philosophy, just my expectation).

That said, it's always worth asking, and the dev mailing list is probably the best way to get some attention.

ijuma added a commit to ijuma/kafka that referenced this pull request Jan 26, 2021
…e-allocations-lz4

* apache-github/trunk: (562 commits)
  MINOR: remove unused code from MessageTest (apache#9961)
  MINOR: Fix visibility of Log.{unflushedMessages, addSegment} methods (apache#9966)
  KAFKA-12229: Restore original class loader in integration tests using EmbeddedConnectCluster during shutdown  (apache#9942)
  KAFKA-12190: Fix setting of file permissions on non-POSIX filesystems (apache#9947)
  MINOR: Remove `toStruct` and `fromStruct` methods from generated protocol classes (apache#9960)
  MINOR: Fix typo in Utils#toPositive (apache#9943)
  MINOR: MessageUtil: remove some deadcode (apache#9931)
  MINOR: Update zstd-jni to 1.4.8-2 (apache#9957)
  MINOR: Revert assertion in MockProducerTest (apache#9956)
  MINOR: Optimize assertions in unit tests (apache#9955)
  MINOR: Tag `RaftEventSimulationTest` as `integration` and tweak it (apache#9925)
  MINOR: Update to Gradle 6.8.1 (apache#9953)
  MINOR: A few small group coordinator cleanups (apache#9952)
  MINOR: Upgrade ducktape to version 0.8.1  (apache#9933)
  MINOR: fix record time in test shouldWipeOutStandbyStateDirectoryIfCheckpointIsMissing (apache#9948)
  MINOR: Restore interrupt status when closing (apache#9863)
  KAFKA-10357: Extract setup of repartition topics from Streams partition assignor (apache#9848)
  KAFKA-12212; Bump Metadata API version to remove `ClusterAuthorizedOperations` fields (KIP-700) (apache#9945)
  MINOR: log 2min processing summary of StreamThread loop (apache#9941)
  MINOR: Drop enable.metadata.quorum config (apache#9934)
  ...
@garyrussell
Copy link
Copy Markdown

@ableegoldman I understand, but for the reasons I stated on the JIRA, this was a critical regression and some users have no choice regarding upgrading.

I have no personal ax to grind; I don't use Windows, but many developers have no choice there either.

I wonder if there could be a new mechanism to just release the kafka-clients, e.g. 2.6.1.1 ?

@ableegoldman
Copy link
Copy Markdown
Member

Yeah that's completely valid, but unfortunately it's not up to me 🙂

I do think you should raise this on the mailing list -- maybe someone there will have an idea of how to proceed. I also think it's worth bringing up to get some discussion going on how we can avoid things like this from happening again, since testing on Windows is (obviously) not part of the current release process.

@notme159
Copy link
Copy Markdown

@wilkinsona Hello thank you for the fix, I still have an issue, using KeyValueStore and getting
org.apache.kafka.streams.processor.internals.StateDirectory - Failed to change permissions for the directory C:\Users\user1\AppData\Local\Temp\kafka-streams
when calling streams = new KafkaStreams(topology, props);

Tried to debug and this always returns false ( set &= file.setWritable(true, true) )
Running win10, admin user, openJdk11 latest, Kafka 3.0.0
The folder has setup Full Control for user1/SYSTEM/Administrators/Everyone, I'm running out of ideas. Thanks for every tip.

@aybefox
Copy link
Copy Markdown

aybefox commented Apr 26, 2022

@notme159
Same here :( Did you find out something?

@notme159
Copy link
Copy Markdown

@aybefox Well the folder permissions are correct (it is writable), it just logs error when the setWritable is called. So I just suppress the StateDirectory log in log4j config using FATAL level only for this class, not to show ERROR in log.
Of course it's not greatest work around as you might suppress another errors in that class.

@BrokenWingsIcarus
Copy link
Copy Markdown

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants