errose28 commented on PR #8871: URL: https://github.com/apache/ozone/pull/8871#issuecomment-3130093683
Thanks for adding a design for this. Based on the discussion in the community sync this morning here are some high level comments: ## Comments on the Specification - Solidifying the schema first will help us work backwards to how we obtain the information - We should decide whether we want to support a modified S3 schema, HDFS inotify schema, or one schema for FSO and one for OBS. - We should define how this approach works with Ranger ACLS - Likely this would just mean that if equivalent functionality was desired for Ranger ACLs, it would need to be incorporated to Ranger outside of Ozone. This is fine but it should be specified in the doc. - The listener implemented for syncing may also be able to run the Ozone Ranger plugin itself and identify ACL changes that way. - We need a way to define event retention. For example, if the notification service is down for x number of days or y number of transactions, all those before that period will be lost. Note that the service can be down for any arbitrary period of time. - Are the notifications cluster-wide, or can they be scoped by bucket or object? - Related to the previous point, how is access control implemented? - Does the consumer of these events needs to authenticate with the cluster to make sure it has permission to see these changes, or is it purely a push model based on server side configs, where the configured endpoint is assumed to be trusted by the one configuring it? - Some example configurations are called out for a Kafka based consumer, but it would probably be helpful to have a pluggable consumer model. - See `IAccessAuthorizer` for one way to do this in the existing Ozone code, where the ACL authorizer can be loaded dynamically from a class provided by a config key. - The "leader execution" project HDDS-11897 will change the format of the Ratis logs drastically. Instead of log contents like `rename key1 t-> key2` the logs will contain DB operations which will be difficult to map back to filesystem events, like `delete key1 from fileTable && put key2 in fileTable`. - For future proofing we should make sure our approach works with both models - If we need to add some extra metadata to the Ratis logs that is only read by the listener to support this, that might be ok. ## Comments on the design - Using a listener node makes it difficult to ensure at-least-once delivery, since that listener node may need to install a Ratis snapshot at any time, losing the incremental Ratis log entries. - The listener node is writing a full RocksDB which is never read. This will require a separate NVMe to keep up with the throughput of the main quorum, since applying to RocksDB is a sequential, blocking operation in Ratis. This incurs hardware overhead that is not really utilized for the task. ### Slight Alternative Approach: Co-locating a listener on each OM I haven't totally worked out this proposal but wanted to put it out here since it seems to address some of the previous concerns. Instead of a full OM, the listener can be a small process that is co-located on each OM node. If its workload ends up being especially light, it may even be able to be a thread within the OM itself. The main OM would be configured to move old Ratis log files to a backup directory instead of deleting them. This keeps its working directory clean and will not affect startup time due to a large number of files. I did a quick look through Ratis and it [doesn't look like this is supported currently](https://github.com/apache/ratis/blob/f21e350d4e7330d1b2f24001ac62040ebba205f4/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java#L483), but it could be added. The listener can read log entries from the backup dir, and then the main OM dir. As a listener, it will be notified of the cluster's apply index, which it can use to determine which log files correspond to valid events. It will also know the current leader through election events, so the instances running on followers can pause running. This listener can then push events to the plugged in consumers based on the Ratis logs, and purge them from the backup dir when the consumers have acke d them. It does not need to consume the ratis logs that come through the Ratis listener API since it will use the local copies on the OM. We would still need to hash out how the at-least-once delivery specification from Ozone to the consumer will fit with leader changes in this model. -- 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...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org