Skip to content

Conversation

@donalmag
Copy link

@donalmag donalmag commented Jul 28, 2025

What changes were proposed in this pull request?

Design doc for an Ozone Manager Notify node to generate Ozone Filesystem event notifications

What is the link to the Apache JIRA

HDDS-13513

How was this patch tested?

N/A

Example configuration to provide a kafka instance:
```xml

<configration>
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe as a future work, make it possible to configure notification on a per bucket basis, which is actually more AWS-like. That way each bucket owner can decide to send notification or send them to different kafka topics, for example.

Copy link
Contributor

Choose a reason for hiding this comment

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

Optionally, support AWS's PutBucketNotificationConfiguration and GetBucketNotificationConfiguration

Copy link
Member

@peterxcli peterxcli left a comment

Choose a reason for hiding this comment

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

We should also consider the multiple OM listener case

  1. Listeners should acquire some kind of lease to prevent they do the same job at the same time.
  2. The lastAppliedTxn progress of messages submission should reach consensus not just persist in OM listener's local, otherwise if the lease is acquired by another listener they wouldn't know the actual progress.

Btw, just FYI, there are some discussion in #8449, you can take it as a reference.

@errose28
Copy link
Contributor

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, 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 acked 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.


Filtering of events or paths/buckets
Persistent storage of notification messages
Asynchronous delivery
Copy link
Contributor

Choose a reason for hiding this comment

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

High availability and consistency.

It sounds like the design does not consider availability. If the Listener goes down there's no guarantee the events notification will be processed.

Consistency: does it guarantee in-order delivery of events? Any delivery latency assumptions or SLAs?

Copy link
Contributor

Choose a reason for hiding this comment

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

Interesting that S3 does not guarantee event order. It seems like this would make it difficult to build any sort of replication based system around these events. We may be able to support this depending how we are working with the Ratis logs.

### At-least-once delivery

At-least-once delivery cannot be guaranteed without some form of persistence, however we want to avoid persisting the notification messages themselves. One approach to achieving this is by "replaying" the proposals.
If we persist the lastAppliedTxn after each notification is sent successfully, on a restart we could reset the lastAppliedTxn on the state machine and replay proposals in a "non-committal" mode in order to generate any missing notifications up to the current proposal.
Copy link
Contributor

Choose a reason for hiding this comment

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

we do persist lastAppliedIndex in ratis log, and there's a callback to the statemachine implementation when a transaction is applied, so it sounds doable.

when we conceived the original design in HDDS-5984, we couldn't do it as S3 gateway is stateless. Making the logic inside the OM follower makes a lot of sense.

@donalmag
Copy link
Author

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, 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 acked 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.

@errose28 - Thanks for the feedback. This approach is interesting as it is similar to a separate POC we had developed internally. The reason we did not go with that approach was that we didn't think we could guarantee the entry in the ratis log was actually applied successfully without attempting to update the metadata itself first. Your comment seems to imply that should be possible though? Can you explain how we could verify this? My understanding is that the lastAppliedTxn is just the last txn the leader executed, there is no guarantee it was executed successfully - is that assumption incorrect? E.g. If the notify process is on txn 2 and the leader is on txn 10 how do we confirm which of txns 2->10 were applied successfully?

@donalmag
Copy link
Author

FYI more detail on the event notification schema has been uploaded in a separate doc

@errose28
Copy link
Contributor

My understanding is that the lastAppliedTxn is just the last txn the leader executed, there is no guarantee it was executed successfully - is that assumption incorrect? E.g. If the notify process is on txn 2 and the leader is on txn 10 how do we confirm which of txns 2->10 were applied successfully?

Yes this is true. Actually the raft-log-only proposal would only work in the context of the new leader execution model in HDDS-11897, where the commit index is guaranteed to have its corresponding operations applied to the state machine. We would then track the commit index to know where in the raft logs we need to generate events for. The current model will update the applied index even for invalid transactions that don't cause mutations like a directory already existing. The challenge is that you need an up to date OM DB to know whether or not a transaction will succeed. However this may require provisioning a whole OM Node with hardware just to push some json messages to an endpoint, and does not have at least once delivery in the presence of Ratis snapshot installs. This overhead would then become tech debt once the leader execution project finishes.

I think the high level design requirements we seem to be converging on are:

  • A ledger of events
  • Highly available persistence for this ledger to maintain at least once delivery
  • Events will need to be persisted before returning to the client, although they may not need to be pushed to the consumers at this point.
  • Minimal extra hardware provisioning

While using Raft logs seems to fit most of these requirements, it is not the only option if other Ratis constraints are getting in the way. Another option is that when the apply happens to the state machine, we write a ledger to the disk if that transaction indicates a successful modification. Basically the ledgers become part of the Ratis state machine, not the Ratis log. The notifier then consumes these ledgers and pushes them to the consumers without worrying about indices, because all ledgers represent successful operations. The notifier can prune these ledgers as it goes. Since we would likely need to add some metadata to the Ratis requests about operation type in HDDS-11897 for most event notification proposals I've seen so far, this should still work for both request models.

Copy link
Contributor

@errose28 errose28 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 adding information on the schema. Based on my current reading I think S3 schema with extensions looks the most promising, but I'll let others weigh in as well.


required fields:
- path (volume + bucket + key)
- isfile
Copy link
Contributor

Choose a reason for hiding this comment

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

Seems like this field would be redundant given the event name. Same with CreateDirectoryRequest.

## SetTimes

event notifications should be raised to inform consumers that
mtime/atime has changed, as per **SetTimesRequest**
Copy link
Contributor

Choose a reason for hiding this comment

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

We don't support atime (only mtime) because atime turns all read operations into write operations which kills performance.

Comment on lines +280 to +282
RenameKeyRequest would be the fromKeyName and the toKeyName of the
*parent* of the directory being renamed (and not the impacted child
objects).
Copy link
Contributor

Choose a reason for hiding this comment

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

Yes this is probably the best way to do it with rename or delete directory generating a single event. These are atomic operations on the Ozone cluster, so ideally the consumer would see them that way as well.

Copy link
Contributor

Choose a reason for hiding this comment

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

For all the request we should also track the bucket layout(OBS/FSO). The consumer event may get some additional info on the kind of event. Behaviour of events like rename and delete are different might be a good info to track.

- no additional processing when emitting events

Cons:
- non-standard S3 event notification semantics
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is acceptable for FSO buckets. FSO buckets accessed through S3 API do compatibility on a best-effort basis, but there are some things that just won't work. For example writing an object called /bucket/prefix then writing another object called /bucket/prefix/key is valid in S3 and OBS but not FSO (the second object would try to create prefix as a directory). IMO it is ok for S3 event notifications to make similar tradeoffs when used with FSO buckets.

NOTE: directory rename is just one example of a hierarchical FSO
operation which impacts child objects. There may be other Ozone
hierarchical FSO operations which will need be catered for in a similar
way (recursive delete?)
Copy link
Contributor

Choose a reason for hiding this comment

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

I think directory delete and rename are the only two that fit this category. We do have atomic recursive directory delete.

Copy link
Contributor

Choose a reason for hiding this comment

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

@errose28 With OM leader execution designing for FSO the from path and toPath might not be defined. I am not sure how we would handle this in case of FSO? In leader execution the gate keeper would do path resolution parallely and only thing valid in that case would be parentId/keyName. I don't think we can really handle this correctly.

Copy link
Contributor

@swamirishi swamirishi Nov 10, 2025

Choose a reason for hiding this comment

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

I believe we should spend some more time what events we want to emit in case of FSO. From what I understand order of event is very important maybe we have to send the entire hierarchy of objectIds witnessed while path resolution so within a ratis transaction batch all rename request we can figure out parents being renamed and the event added within a ratis batch has to be ordered.
For instance within a ratis batch say we have:
d1(O1)/d2(O2)/d3(O3)/d4(O4)/F1(O5)

Now within a ratis transaction batch if I have 2 parallel transactions:
mv d1/d2 -> d1/d5
mv d1/d2/d3/d4/F1 -> d1/d2/d3/F1

then notification should be either
mv d1/d2/d3/F1 -> d1/d2/F1 and mv d1/d2 -> d1/d5
or
mv d1/d2 -> d1/d5 and mv d1/d5/d3/F1 -> d1/d5/F1

mv d1/d2 -> d1/d5 and mv d1/d2/d3/F1 -> d1/d2/F1 would be invalid

So a ratis txn batch should send the entire objectId hierarchy in the batch request to figure out this change and identify all the paths transformation this could get a bit complex I had experienced this first hand when we were implementing snapshot diff and we had scrapped the idea of making snaphshot diff order compliant but I believe this event notification design cannot be agnostic to order of events here.


Filtering of events or paths/buckets
Persistent storage of notification messages
Asynchronous delivery
Copy link
Contributor

Choose a reason for hiding this comment

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

Interesting that S3 does not guarantee event order. It seems like this would make it difficult to build any sort of replication based system around these events. We may be able to support this depending how we are working with the Ratis logs.


## Performance

While this is a synchronous only approach any latency between notification target and OM should not impact the performance of client requests as the notification does not run on the leader.
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure "synchronous" is the best term to use here. Ceph's definitions are that synchronous blocks the writer and asynchronous does not. It's probably clearer to use similar definitions here, so the system would be asynchronous. I think the point being made here is that the notifier will block until the consumer acks the event but that this will not block the writer.

@errose28
Copy link
Contributor

errose28 commented Aug 5, 2025

@jojochuang
Copy link
Contributor

There was a healthy discussion on this topic today. Can any one summarize what we plan to do (or at least where we think we're converging onto?)

@errose28
Copy link
Contributor

Here is the recording of today's discussion:
https://cloudera.zoom.us/rec/share/gV55URMxUdXpdq8UYHy4EtyoZ3a1Vro1lJYUWeQDA5smHHuVLyr4K7zqY5ddgbIK.h5bSNPYJvhDEbwaV
Passcode: jj6oSj4?

Here is a high level summary of what was discussed. Feel free to add other details I may have missed.


The high level design discussed involves the Ozone Manager persisting a bounded number of events to RocksDB as they occur. Plugins decoupled from the OM can consume these events by iterating the RocksDB column family which stores these events. Plugins would have freedom to make many decisions as needed for their use case:

  • How they want to persist the index of the event they last left off on
  • Which node should have the currently active plugin
  • Whether the plugin pushes to a single source or multiple sources
  • Whether event delivery to the source is blocking or non-blocking
  • What schema they want to output

Internally, the Ozone Manager can persist events to its state machine as part of the applyTransaction execution of relevant requests. This ensures high availability of events for at least once delivery. Events can be written to a new column family in RocksDB using a key based on an incrementing transaction ID. The OM would have a configuration like ozone.om.events.max which would define the maximum number of events that would remain persisted. Once this number is exceeded, older events may be deleted by the OM with a RocksDB range delete.

This provides a spec for plugins consuming events: They can be guaranteed to see at least the last ozone.om.max.events, and they can use the transaction ID of the last event they processed to resume processing newer events.

To run plugins, a few options were discussed:

  1. Run each plugin as a thread within the OM
    • Pros:
      • Easiest process and configuration management
      • Easy for plugins to check if the current OM is the leader, and use this as their run condition
      • Plugins can add metrics to the OM's JMX endpoint for monitoring.
    • Cons:
      • Potentially more load on the OM if there are a lot of plugins pushing information or churning iterator objects.
      • Adding or removing plugins would likely require (rolling) restart of OMs since the classpath must be updated
      • If a plugin's thread crashes unexpectedly, either the OM must be restarted to revive it, or a management CLI to start or stop plugins must be added.
  2. Run each plugin as a process on each OM using a read-only RocksDB handle to consume events.
    • Pros:
      • No extra load on the OM process.
      • Plugins can be added or removed at any time independent of all other plugins or OM.
    • Cons:
      • Process management complexity increases with number of plugins.
      • Cannot do a local leader check if that is to be used as a run condition. Best option is probably to query metrics of the current host to check its leader status.
      • Would need to add a new JMX endpoint if plugin metrics were to be added.
  3. Run one process on each OM that internally runs all configured plugins
    • Mostly the same pros and cons as the previous option.
    • Pros:
      • Less process management than the second option, while still removing all load from the OM.
      • Plugins can be added, removed, or restarted independent of the OM, but not other plugins.
    • Cons:
      • Still need to manage a separate process with its own configuration and metrics.
      • Cannot do a local leader check, still need to rely on the host OM's metrics.

For either approach, we could create a base plugin class that would handle most of the common tasks. The OM already supports dynamically loading plugins that implement a given interface like IAccessAuthorizer. We can use a similar model here:

  • OMEventListener (or other name): The interface that all plugins must implement. It would probably just contain a single run method that would be started with the process and have complete autonomy over how it reads and processes events.
  • OMEventListenerBase (or other name): An abstract class that provides a common starting implementation that most plugins would likely want to use for consuming events:
    • Running a thread that fetches all the events from the DB from the last processed index, deserializing them, and invoking a callback to the implementation to consume them.
    • Periodically persisting the last event index that was processed.
      • This does not need to happen for every event due to at least once delivery.
      • Persistence options include:
        • Writing to a local file (harder to resume work on leader change)
        • Writing the information to a file in Ozone
    • Checking if the current node is the leader to determine whether the plugin should continue pushing events or sleep
      • This failover mechanism also does not need to be exact due to at least once delivery. A follower may continue to deliver events before realizing it is no longer the leader without violating this protocol.

The biggest open item at the end of this discussion was whether to run the plugins within the OM as threads or as a separate process.

@gardenia
Copy link

We are just updating the design doc based on the latest feedback and the discussion from last week's community call. I have realized I was left with some uncertainty over the ledger schema which I thought would be good to flesh out here before re-publishing the design doc.

My original strawman schema as discussed in last week's call:

 message OperationInfo {
  optional int64 trxLogIndex = 1;
  optional string op = 2;
  optional string volumeName = 3;
  optional string bucketName = 4;
  optional string keyName = 5;
  optional uint64 creationTime = 6;
  repeated hadoop.hdds.KeyValue extraArgs = 7;
}

@errose28 made a good point that the use of the extraArgs field (effectively a Map<String, String>) for miscellaneous fields (such as the rename "to" path or acl fields) would lead to a brittle schema management.

@errose28 you referenced a pattern where such things can be provided as a bunch of "optional" fields (for examples, as per OMRequest) but it wasn't clear exactly what you had in mind therefore I thought it would be useful to discuss a few possible intrepretations:

  1. use the full request object in the ledger item schema, e.g. :
message OperationInfo {                                                                                       
  required int64 trxLogIndex = 1;                                                                             
                                                                                                              
  required Type cmdType = 2; // Type of the command                                                           
  required uint64 creationTime = 3;                                                                           
                                                                                                              
  optional CreateKeyRequest          createKeyRequest        = 4;                                             
  optional RenameKeyRequest          renameKeyRequest        = 5;                                             
  optional DeleteKeyRequest          deleteKeyRequest        = 6;                                             
  optional CommitKeyRequest          commitKeyRequest        = 7;                                             
  optional CreateDirectoryRequest    createDirectoryRequest  = 8;                                             
  optional CreateFileRequest         createFileRequest       = 9;                                             
}

initially this is what I inferred that you meant but I may have misunderstood. At any rate - I'm not sure this is ideal - for a couple of reasons:
a. it pulls in the entire request object into the ledger which may be overkill
b. it is a bit loose about what it provides to consumers of the ledger and so those consumers could end up using fields they shouldn't and end up depending on them.

  1. a less loose interpretation
message CreateKeyOperationArgs {                                                                              
}                                                                                                             
                                                                                                              
message RenameKeyOperationArgs {                                                                              
    required string toKeyName = 1;                                                                            
}                                                                                                             
                                                                                                              
message DeleteKeyOperationArgs {                                                                              
}                                                                                                             
                                                                                                              
message CommitKeyOperationArgs {                                                                              
}                                                                                                             
                                                                                                              
message CreateDirectoryOperationArgs {                                                                        
}                                                                                                             
                                                                                                              
message CreateFileOperationArgs {                                                                             
    required bool isRecursive = 2;                                                                            
    required bool isOverwrite = 3;                                                                            
}                                                                                                             
                                                                                                                                                                                                                  
message OperationInfo {                                                                                       
                                                                                                              
  optional int64 trxLogIndex = 1;                                                                             
  required Type cmdType = 2; // Type of the command                                                           
  optional string volumeName = 3;                                                                             
  optional string bucketName = 4;                                                                             
  optional string keyName = 5;                                                                                
  optional uint64 creationTime = 6;                                                                           
                                                                                                              
  optional CreateKeyOperationArgs       createKeyArgs = 7;                                                    
  optional RenameKeyOperationArgs       renameKeyArgs = 8;                                                    
  optional DeleteKeyOperationArgs       deleteKeyArgs = 9;                                                    
  optional CommitKeyOperationArgs       commitKeyArgs = 10;                                                   
  optional CreateDirectoryOperationArgs createDirectoryArgs = 11;                                             
  optional CreateFileOperationArgs      createFileArgs = 12;                                                  
}

This is effectively the same "optional" trick but rather than copy in the full request (from the OmRequest) we cherry-pick the fields which are exposed to the ledger via explicitly defined helper messages (RenameKeyOperationArgs, CreateFileOperationArgs etc) and so consumers have a clear contract as to what they can consume. This provides stronger typing around what extra fields are provided and this also makes schema changes simpler.

It could be argued that #1 gives more flexibility to consumers of the ledger but I feel like a more explicit contract (as per #2) is ultimately better and to keep it minimal to begin with makes sense. Therefore I would have a preference for #2.

But I would like to verify that I correctly understood your feedback @errose28 so please let me know what you (and others) think (or if you have a different suggestion).

@errose28
Copy link
Contributor

errose28 commented Sep 8, 2025

Discussed in the community sync today. We agreed approach 2 for proto schema looks better since it allows us to choose which fields to expose to consumers.

Copy link
Contributor

@errose28 errose28 left a comment

Choose a reason for hiding this comment

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

This design proposal LGTM. Exact specifics on items like names of fields in the proto schema and metrics can be worked out in the implementation phase with more context. I will also take a look at the prototype in #9012 for some high level suggestions. From there we can work on splitting the prototype down into mergable pieces which we can review in more detail.

It would be good to have others sign off on this approach as well before we put code into a branch. Tagging @jojochuang @swamirishi and @sumitagrawl since they have been present in the community meetings where this was discussed, but others please check it out as well.

@ivandika3 ivandika3 self-requested a review November 1, 2025 04:38

[https://docs.aws.amazon.com/AmazonS3/latest/userguide/notification-how-to-event-types-and-destinations.html#supported-notification-event-types](https://docs.aws.amazon.com/AmazonS3/latest/userguide/notification-how-to-event-types-and-destinations.html#supported-notification-event-types)

has become a standard for change notifications in S3 compatible storage services such as S3 itself, Ceph, MinIO etc
Copy link
Contributor

@ivandika3 ivandika3 Nov 7, 2025

Choose a reason for hiding this comment

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

(Just a passing comment) In the future, we can also support CloudEvents (https://cloudevents.io/) which is also supported by major cloud provider (AWS EventBridge, GCP Eventarc, Azure Event Grid) and ASF project like EventMesh (https://eventmesh.apache.org/docs/design-document/event-handling-and-integration/cloudevents/)

Copy link
Contributor

Choose a reason for hiding this comment

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

Nice, I haven't seen this before. With the plugin model we should be able to support any format as needed, and users can bring their own formats if those that end up shipping with Ozone don't cover their needs.

Choose a reason for hiding this comment

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

Can we adopt Cloud event format from the start?

Copy link
Contributor

@swamirishi swamirishi left a comment

Choose a reason for hiding this comment

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

@donalmag thank you for the design doc I have a few doubts, concerns and suggestions in the design would love if we can iron out the last mile details here.

we should emit some **create** event

required fields:
- path (volume + bucket + key)
Copy link
Contributor

Choose a reason for hiding this comment

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

For all the request we should also track the bucket layout(OBS/FSO). The consumer event may get some additional info on the kind of event. Behaviour of events like rename and delete are different might be a good info to track.

Comment on lines +280 to +282
RenameKeyRequest would be the fromKeyName and the toKeyName of the
*parent* of the directory being renamed (and not the impacted child
objects).
Copy link
Contributor

Choose a reason for hiding this comment

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

For all the request we should also track the bucket layout(OBS/FSO). The consumer event may get some additional info on the kind of event. Behaviour of events like rename and delete are different might be a good info to track.

NOTE: directory rename is just one example of a hierarchical FSO
operation which impacts child objects. There may be other Ozone
hierarchical FSO operations which will need be catered for in a similar
way (recursive delete?)
Copy link
Contributor

Choose a reason for hiding this comment

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

@errose28 With OM leader execution designing for FSO the from path and toPath might not be defined. I am not sure how we would handle this in case of FSO? In leader execution the gate keeper would do path resolution parallely and only thing valid in that case would be parentId/keyName. I don't think we can really handle this correctly.

NOTE: directory rename is just one example of a hierarchical FSO
operation which impacts child objects. There may be other Ozone
hierarchical FSO operations which will need be catered for in a similar
way (recursive delete?)
Copy link
Contributor

@swamirishi swamirishi Nov 10, 2025

Choose a reason for hiding this comment

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

I believe we should spend some more time what events we want to emit in case of FSO. From what I understand order of event is very important maybe we have to send the entire hierarchy of objectIds witnessed while path resolution so within a ratis transaction batch all rename request we can figure out parents being renamed and the event added within a ratis batch has to be ordered.
For instance within a ratis batch say we have:
d1(O1)/d2(O2)/d3(O3)/d4(O4)/F1(O5)

Now within a ratis transaction batch if I have 2 parallel transactions:
mv d1/d2 -> d1/d5
mv d1/d2/d3/d4/F1 -> d1/d2/d3/F1

then notification should be either
mv d1/d2/d3/F1 -> d1/d2/F1 and mv d1/d2 -> d1/d5
or
mv d1/d2 -> d1/d5 and mv d1/d5/d3/F1 -> d1/d5/F1

mv d1/d2 -> d1/d5 and mv d1/d2/d3/F1 -> d1/d2/F1 would be invalid

So a ratis txn batch should send the entire objectId hierarchy in the batch request to figure out this change and identify all the paths transformation this could get a bit complex I had experienced this first hand when we were implementing snapshot diff and we had scrapped the idea of making snaphshot diff order compliant but I believe this event notification design cannot be agnostic to order of events here.

1. Add a new RocksDB column family e.g. om_event_log.
2. Add a hook in the OMRequest execution workflow (post successful commit) to persist required events.
3. Implement a plugin framework to run notification publishers.
4. Implement a new background service for cleaning the events table, similar to KeyDeletingService, which operates
Copy link
Contributor

Choose a reason for hiding this comment

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

No please don't add another ratis request and a background service in om unnecessarily. This can be done within the same om ratis transaction using rocksdb deleteRange API. as long as the column family key is going to ordered in the same order as the trasaction id. This would be a very cheap operation as this would just add another tombstone to rocksdb which should be ok
Please look at this implementation
https://github.com/apache/ozone/pull/8779/files#r2510853726

## Performance

Writes to the RocksDB table happen synchronously in the OM Commit path but are a single put operation.
Deletes are to be executed by the OM in a separate thread ensuring the table is bounded to a specified limit.
Copy link
Contributor

Choose a reason for hiding this comment

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

We have faced issues in DelegationToken implementation in the past because of the divergence. We don't want the event notification be another cause for this. Look at the comment above which has the way we can handle this.

@swamirishi
Copy link
Contributor

@adoroszlai
Copy link
Contributor

use this Feature Branch https://github.com/apache/ozone/tree/HDDS-13513_Event_Notification_FeatureBranch

@swamirishi if you push and then delete a feature branch, please make sure to cancel the corresponding workflow run.

@adoroszlai
Copy link
Contributor

use this Feature Branch https://github.com/apache/ozone/tree/HDDS-13513_Event_Notification_FeatureBranch

@swamirishi if you push and then delete a feature branch, please make sure to cancel the corresponding workflow run.

But not the run for the feature branch you intend to keep.


nice to have fields:
- recursive (if known)

Choose a reason for hiding this comment

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

it would be nice to have 'Size to be reclaimed'

{"Records":[
{
"eventVersion":"2.1",
"eventSource":"ceph:s3",

Choose a reason for hiding this comment

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

should it be "ozon:s3"?

{
"eventVersion":"2.1",
"eventSource":"ceph:s3",
"awsRegion":"us-east-1",

Choose a reason for hiding this comment

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

how do we set regions for private deployments?

@github-actions
Copy link

github-actions bot commented Dec 9, 2025

This PR has been marked as stale due to 21 days of inactivity. Please comment or remove the stale label to keep it open. Otherwise, it will be automatically closed in 7 days.

@github-actions github-actions bot added the stale label Dec 9, 2025
Includes event notification schema design
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.

10 participants