swamirishi commented on code in PR #8871:
URL: https://github.com/apache/ozone/pull/8871#discussion_r2510854640


##########
hadoop-hdds/docs/content/design/event-notifications.md:
##########
@@ -0,0 +1,257 @@
+---
+title: Event notification support in Ozone
+summary: Event notifications for all bucket/event types in ozone
+date: 2025-06-28
+jira: HDDS-13513
+status: design
+author: Donal Magennis, Colm Dougan
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Abstract
+
+Implement an event notification system for Apache Ozone, providing the ability 
for users to consume events occurring on the Ozone filesystem.
+This is similar to https://issues.apache.org/jira/browse/HDDS-5984 but aims to 
encapsulate all events and not solely S3 buckets.  
+This document proposes a potential solution and discusses some of the 
challenges/open questions.
+
+## Introduction
+
+Apache Ozone does not currently provide the ability to consume filesystem 
events, similar to how HDFS does with Inotify or S3 with bucket notifications.  
+These events are an integral part of integration with external systems to 
support real-time, scalable, and programmatic monitoring of changes in the data 
or metadata stored in Ozone.  
+These external systems can use notifications of objects created/deleted to 
trigger data processing workflows, replication and monitoring alerts.
+
+### Goals
+
+Durable event log within each OM containing relevant OMRequest information for 
notification purposes.
+Plugin framework for publishers (e.g. Kafka/RabbitMQ, custom sinks) running in 
separate threads in the OM.
+Provide support for all events across the Ozone filesystem for FSO and non FSO 
buckets, including renames and changes to acls.
+Guarantee at-least-once delivery within a bounded retention period, with 
notification of "missed events" where applicable.
+Read-only access for plugins to notification table.
+
+### Non-Goals
+
+Exactly-once end-to-end semantics to external systems.
+Filtering of events or paths/buckets.
+Cross-OM consensus about what has been notified; co-ordination to be defined 
in the plugin e.g. write last notified position to a file in Ozone.
+Retrofitting historical events prior to feature enablement.
+
+### Supported OMRequests
+
+OMDirectoryCreateRequest
+OMKeyCommitRequest
+OMKeyDeleteRequest
+OMKeyRenameRequest
+OMKeyAddAclRequest
+OMKeyRemoveAclRequest
+OMKeySetAclRequest
+OMKeySetTimesRequest
+
+# Design
+
+## Overview
+
+Introduce an Event Notification Pipeline for Apache Ozone with two
+logical pieces:
+
+1. event data capture
+
+* OM captures the required details of selected OMRequest write
+  operations post metadata update and persists them to a dedicated RocksDB
+  completed operations "ledger" table keyed by the Ratis Txn Id
+* each OM independently produces items to its local ledger table.  The
+  ledger table should be integrated into OM Snapshots so that all OM's
+  converge on the full set of required notifications.
+* a retention policy is to be implemented in order to clean up no longer 
required entries.  This policy is bounded to a table size(number of events) 
which can be configurable.
+* event capture will only be enabled if enabled
+
+2. event data publishing
+
+* a plugin framework is exposed where plugins can consume the ledger
+  items in read-only fashion and process them as desired
+* Plugins will run inside the OM and should be cognisant of resource 
consumption i.e. memory/disk
+* all OMs will run the plugins but only the current leader OM will be
+  active
+* a base plugin implementation will provide common behaviour, including
+  read only iteration of new ledger items and flagging that events
+  have been "missed" since the consumer last requested them
+  leader OM will be active
+* a concrete plugin implementation will deal with publishing
+  notifications to external targets (Apache Kafka)
+
+### Components
+
+#### Ozone Manager
+
+Changes are required in the OzoneManager:
+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

Review Comment:
   
   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



##########
hadoop-hdds/docs/content/design/event-notification-schema.md:
##########
@@ -0,0 +1,396 @@
+---
+title: Event notification schema discussion
+summary: Event notifications schema discussion
+date: 2025-06-29
+jira: HDDS-13513
+status: design
+author: Colm Dougan, Donal Magennis
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+## Overview
+
+This document outlines the schema requirements for event notification
+within Ozone and discusses the suitability of 2 widely used event
+notification schemas (S3 and HDFS) as candidates to use as a basis for
+the transmission format for notifications within Ozone.
+
+# General schema requirements
+
+## File/Directory creation/modification
+
+event notifications should be raised to inform consumers of completed
+operations which modify the filesystem and specifically the requests:
+
+#### CreateRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateFileRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)

Review Comment:
   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.



##########
hadoop-hdds/docs/content/design/event-notifications.md:
##########
@@ -0,0 +1,257 @@
+---
+title: Event notification support in Ozone
+summary: Event notifications for all bucket/event types in ozone
+date: 2025-06-28
+jira: HDDS-13513
+status: design
+author: Donal Magennis, Colm Dougan
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Abstract
+
+Implement an event notification system for Apache Ozone, providing the ability 
for users to consume events occurring on the Ozone filesystem.
+This is similar to https://issues.apache.org/jira/browse/HDDS-5984 but aims to 
encapsulate all events and not solely S3 buckets.  
+This document proposes a potential solution and discusses some of the 
challenges/open questions.
+
+## Introduction
+
+Apache Ozone does not currently provide the ability to consume filesystem 
events, similar to how HDFS does with Inotify or S3 with bucket notifications.  
+These events are an integral part of integration with external systems to 
support real-time, scalable, and programmatic monitoring of changes in the data 
or metadata stored in Ozone.  
+These external systems can use notifications of objects created/deleted to 
trigger data processing workflows, replication and monitoring alerts.
+
+### Goals
+
+Durable event log within each OM containing relevant OMRequest information for 
notification purposes.
+Plugin framework for publishers (e.g. Kafka/RabbitMQ, custom sinks) running in 
separate threads in the OM.
+Provide support for all events across the Ozone filesystem for FSO and non FSO 
buckets, including renames and changes to acls.
+Guarantee at-least-once delivery within a bounded retention period, with 
notification of "missed events" where applicable.
+Read-only access for plugins to notification table.
+
+### Non-Goals
+
+Exactly-once end-to-end semantics to external systems.
+Filtering of events or paths/buckets.
+Cross-OM consensus about what has been notified; co-ordination to be defined 
in the plugin e.g. write last notified position to a file in Ozone.
+Retrofitting historical events prior to feature enablement.
+
+### Supported OMRequests
+
+OMDirectoryCreateRequest
+OMKeyCommitRequest
+OMKeyDeleteRequest
+OMKeyRenameRequest
+OMKeyAddAclRequest
+OMKeyRemoveAclRequest
+OMKeySetAclRequest
+OMKeySetTimesRequest
+
+# Design
+
+## Overview
+
+Introduce an Event Notification Pipeline for Apache Ozone with two
+logical pieces:
+
+1. event data capture
+
+* OM captures the required details of selected OMRequest write
+  operations post metadata update and persists them to a dedicated RocksDB
+  completed operations "ledger" table keyed by the Ratis Txn Id
+* each OM independently produces items to its local ledger table.  The
+  ledger table should be integrated into OM Snapshots so that all OM's
+  converge on the full set of required notifications.
+* a retention policy is to be implemented in order to clean up no longer 
required entries.  This policy is bounded to a table size(number of events) 
which can be configurable.
+* event capture will only be enabled if enabled
+
+2. event data publishing
+
+* a plugin framework is exposed where plugins can consume the ledger
+  items in read-only fashion and process them as desired
+* Plugins will run inside the OM and should be cognisant of resource 
consumption i.e. memory/disk
+* all OMs will run the plugins but only the current leader OM will be
+  active
+* a base plugin implementation will provide common behaviour, including
+  read only iteration of new ledger items and flagging that events
+  have been "missed" since the consumer last requested them
+  leader OM will be active
+* a concrete plugin implementation will deal with publishing
+  notifications to external targets (Apache Kafka)
+
+### Components
+
+#### Ozone Manager
+
+Changes are required in the OzoneManager:
+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
+in a deterministic manner across all OM's
+
+#### Plugin Framework
+
+Plugin Manager - spawns and supervises plugin threads.
+
+Base Plugin - Provides common functionality which can be re-used:
+1. Leader check.
+2. Read-only query on the RocksDb table.
+3. Offset tracking and persistence.
+4. MissedEvents notifications.
+
+It should be possible to run multiple plugins at the same time.
+
+Plugins should be configured such that the implementation can be loaded
+if provided on the classpath, similarly to ranger plugins which
+are configured as follows:
+
+```
+    xasecure.audit.destination.kafka=true
+    
xasecure.audit.destination.kafka.classname=org.apache.ranger.audit.provider.kafka.KafkaAuditProvider
+    xasecure.audit.kafka.topic_name=ranger_audits
+    xasecure.audit.kafka.other_config_key=abc123
+```
+
+#### RocksDB Table
+
+The ledger will be stored as a RocksDb column family where the
+transaction id of the successful write operation is the key and the
+value is an object with the folliwng sample protobuf schema:
+
+```
+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;
+}
+```
+
+## 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.

Review Comment:
   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.



##########
hadoop-hdds/docs/content/design/event-notification-schema.md:
##########
@@ -0,0 +1,396 @@
+---
+title: Event notification schema discussion
+summary: Event notifications schema discussion
+date: 2025-06-29
+jira: HDDS-13513
+status: design
+author: Colm Dougan, Donal Magennis
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+## Overview
+
+This document outlines the schema requirements for event notification
+within Ozone and discusses the suitability of 2 widely used event
+notification schemas (S3 and HDFS) as candidates to use as a basis for
+the transmission format for notifications within Ozone.
+
+# General schema requirements
+
+## File/Directory creation/modification
+
+event notifications should be raised to inform consumers of completed
+operations which modify the filesystem and specifically the requests:
+
+#### CreateRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateFileRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateDirectoryRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+#### CommitKeyRequest
+
+we should emit some **commit/close** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- data size
+- hsync?
+
+#### DeleteKeyRequest
+
+we should emit some **delete** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- recursive (if known)
+
+### RenameKeyRequest
+
+we should emit some **rename** event
+
+required fields:
+- fromPath (volume + bucket + key)
+- toPath (volume + bucket + toKeyName)
+
+nice to have fields:
+- recursive (if known)
+- is directory (if known)
+
+NOTE: in the case of a FSO directory rename there is a dillema
+(discussed later in this document) as to whether we should emit a single
+event for a directory rename (specifying only the old/new directory names)
+or whether we should emit granular events for all the child objects impacted by
+the rename.
+
+## ACLs
+
+event notifications should be raised to inform consumers that ACL events
+have happened. The relevant requests are:
+
+* AddAclRequest
+* SetAclRequest
+* RemoveAclRequest
+
+The fields provided could vary based on the implementation complexity.
+
+Minimally we have a requirement that we be informed that "some ACL update
+happened" to a certain key (or prefix).
+
+Ideally the details would include the full context of the change made as
+per the request. (perhaps by mirroring the full request details as a JSON
+sub-object) e.g. :
+
+```json
+   ...
+
+   "acls": [
+    {
+      type: "GROUP",
+      name: "mygroup"
+      rights: "\000\001",
+      aclScope: "ACCESS",
+    }
+   ]
+```
+
+The precise details we would need to revisit with guidance from the
+community but this is just to set broad brush expectations.
+
+## SetTimes
+
+event notifications should be raised to inform consumers that
+mtime/atime has changed, as per **SetTimesRequest**
+
+# Transmission format
+
+This section discusses 2 widely used transmission formats for event
+notifiations (S3 and HDFS) and their suitability as candidates for
+adoption within Ozone.
+
+It is not assumed that these are the only options available but they are
+good examples to test against our requirements and discuss trade-offs.
+
+## 1. S3 Event Notification schema
+
+The S3 event notification schema:
+
+[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
+
+Notification events are produced as a list of JSON records.
+
+To illustrate we can look at a sample "create" event from the Ceph docs
+(https://docs.ceph.com/en/quincy/radosgw/notifications/#events):
+
+```json
+
+{"Records":[
+    {
+        "eventVersion":"2.1",
+        "eventSource":"ceph:s3",
+        "awsRegion":"us-east-1",
+        "eventTime":"2019-11-22T13:47:35.124724Z",
+        "eventName":"ObjectCreated:Put",
+        "userIdentity":{
+            "principalId":"tester"
+        },
+        "requestParameters":{
+            "sourceIPAddress":""
+        },
+        "responseElements":{
+            
"x-amz-request-id":"503a4c37-85eb-47cd-8681-2817e80b4281.5330.903595",
+            "x-amz-id-2":"14d2-zone1-zonegroup1"
+        },
+        "s3":{
+            "s3SchemaVersion":"1.0",
+            "configurationId":"mynotif1",
+            "bucket":{
+                "name":"mybucket1",
+                "ownerIdentity":{
+                    "principalId":"tester"
+                },
+                "arn":"arn:aws:s3:us-east-1::mybucket1",
+                "id":"503a4c37-85eb-47cd-8681-2817e80b4281.5332.38"
+            },
+            "object":{
+                "key":"myimage1.jpg",
+                "size":"1024",
+                "eTag":"37b51d194a7513e45b56f6524f2d51f2",
+                "versionId":"",
+                "sequencer": "F7E6D75DC742D108",
+                "metadata":[],
+                "tags":[]
+            }
+        },
+        "eventId":"",
+        "opaqueData":"[email protected]"
+    }
+]}
+```
+
+As we can see above: there are a number of boilerplate fields to inform us
+of various aspects of the completed operation but there are a few fundamental
+aspects to highlight;
+
+1. the "key" informs us of the key that the operation was performed on.
+
+2. the "eventName" informs us of the type of operation that was
+   performed.  The 2 most notable eventNames are **ObjectCreated:Put** and
+   **ObjectRemoved:Deleted** which pertain to key creation and deletion 
respectively.
+
+3. operation specific fields can be included within the "object" sub-object (in
+   the above example we can see that "size" and "eTag" of the created object 
are included)
+
+## Applicability to Ozone
+
+For non-FSO Ozone buckets / operations there is a clear mapping between
+operations such as CreateKey / CommitKey / DeleteKey / RenameKey and the
+standard S3 event notification semantics.
+
+Examples:
+
+1. CommitKey could be mapped to a ObjectCreated:Put "/path/to/keyToCreate" 
notification event
+
+2. DeleteKey could be mapped to a ObjectRemoved:Deleted "/path/to/keyToDelete" 
notification event
+
+3. RenameKey (assuming a file based key) in standard S3 event noification 
semantics would produce 2 events:
+
+- a ObjectRemoved:Deleted event for the source path of the rename
+- a ObjectCreated:Put event for the destination path of the rename
+
+The challenge in adopting S3 Event notification semantics within Ozone
+would be in at least 2 areas:
+
+### 1. FSO hierarchical operations which impact multiple child keys
+
+Example: directory renames
+
+To illustrate with an example: lets say we have the following simple directory 
structure:
+
+```
+  /vol1/bucket1/myfiles/f1
+  /vol1/bucket1/myfiles/f2
+  /vol1/bucket1/myfiles/subdir/f1
+```
+
+If a user performs a directory rename such as:
+
+```
+  ozone fs -mv /vol1/bucket1/myfiles /vol1/bucket1/myfiles-RENAMED
+```
+
+Within standard S3 event notification semantics we would expect to see 6 
notifications
+emitted in that case:
+
+```
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f1
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f2
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/subdir/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f2
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/subdir/f1
+```
+
+However, with an approach of simply producing notifications based on Ratis
+state machine events then all we would have to go on from the
+RenameKeyRequest would be the fromKeyName and the toKeyName of the
+*parent* of the directory being renamed (and not the impacted child
+objects).
+
+Therefore to produce notifications using the standard S3 event
+notification semantics for FSO directory renames we would need to
+consider the trade-offs between compatibility with the normal S3
+semantics for renames vs a custom event type for directory renames.
+
+### most compatible approach
+
+We could introduce some additional processing before emitting notification
+events in the case of a directory rename which "gathers together" (prior
+to the change being committed to the DB) the child objects impacted by
+the directory rename and emits pairs of delete/create events for each
+key (as described above)
+
+Pros:
+- standard S3 event notification rename semantics
+
+Cons:
+- additional processing to pull together the events.  This could mean an
+  unknown amount of additional processing for large directory renames.
+- could be a performance drag if performed on the leader
+
+### custom event type
+
+Conversely - we could opt to not try to be fully compliant with existing S3 
event notification
+semantics since the schema was designed for non-hierarchical filesystems and
+instead create some custom event extension (e.g. ObjectRenamed:) and
+emit just a single event for directory renames which specifies only the parent
+paths impacted by the rename:
+
+e.g.
+```
+  eventName=ObjectReanmed:Reanmed, fromKey=myfiles, toKey=myfiles-RENAMED
+```
+
+.. it would then be up to the notification consumer to deal with the
+different rename event semantics (i.e. that only the parent names were
+notified and not the impacted child objects).
+
+This is the same semantics used in the HDFS inotify directory rename
+event (see below).
+
+Pros:
+- no additional processing when emitting events
+
+Cons:
+- non-standard S3 event notification semantics
+
+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?)

Review Comment:
   @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.



##########
hadoop-hdds/docs/content/design/event-notification-schema.md:
##########
@@ -0,0 +1,396 @@
+---
+title: Event notification schema discussion
+summary: Event notifications schema discussion
+date: 2025-06-29
+jira: HDDS-13513
+status: design
+author: Colm Dougan, Donal Magennis
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+## Overview
+
+This document outlines the schema requirements for event notification
+within Ozone and discusses the suitability of 2 widely used event
+notification schemas (S3 and HDFS) as candidates to use as a basis for
+the transmission format for notifications within Ozone.
+
+# General schema requirements
+
+## File/Directory creation/modification
+
+event notifications should be raised to inform consumers of completed
+operations which modify the filesystem and specifically the requests:
+
+#### CreateRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateFileRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateDirectoryRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+#### CommitKeyRequest
+
+we should emit some **commit/close** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- data size
+- hsync?
+
+#### DeleteKeyRequest
+
+we should emit some **delete** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- recursive (if known)
+
+### RenameKeyRequest
+
+we should emit some **rename** event
+
+required fields:
+- fromPath (volume + bucket + key)
+- toPath (volume + bucket + toKeyName)
+
+nice to have fields:
+- recursive (if known)
+- is directory (if known)
+
+NOTE: in the case of a FSO directory rename there is a dillema
+(discussed later in this document) as to whether we should emit a single
+event for a directory rename (specifying only the old/new directory names)
+or whether we should emit granular events for all the child objects impacted by
+the rename.
+
+## ACLs
+
+event notifications should be raised to inform consumers that ACL events
+have happened. The relevant requests are:
+
+* AddAclRequest
+* SetAclRequest
+* RemoveAclRequest
+
+The fields provided could vary based on the implementation complexity.
+
+Minimally we have a requirement that we be informed that "some ACL update
+happened" to a certain key (or prefix).
+
+Ideally the details would include the full context of the change made as
+per the request. (perhaps by mirroring the full request details as a JSON
+sub-object) e.g. :
+
+```json
+   ...
+
+   "acls": [
+    {
+      type: "GROUP",
+      name: "mygroup"
+      rights: "\000\001",
+      aclScope: "ACCESS",
+    }
+   ]
+```
+
+The precise details we would need to revisit with guidance from the
+community but this is just to set broad brush expectations.
+
+## SetTimes
+
+event notifications should be raised to inform consumers that
+mtime/atime has changed, as per **SetTimesRequest**
+
+# Transmission format
+
+This section discusses 2 widely used transmission formats for event
+notifiations (S3 and HDFS) and their suitability as candidates for
+adoption within Ozone.
+
+It is not assumed that these are the only options available but they are
+good examples to test against our requirements and discuss trade-offs.
+
+## 1. S3 Event Notification schema
+
+The S3 event notification schema:
+
+[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
+
+Notification events are produced as a list of JSON records.
+
+To illustrate we can look at a sample "create" event from the Ceph docs
+(https://docs.ceph.com/en/quincy/radosgw/notifications/#events):
+
+```json
+
+{"Records":[
+    {
+        "eventVersion":"2.1",
+        "eventSource":"ceph:s3",
+        "awsRegion":"us-east-1",
+        "eventTime":"2019-11-22T13:47:35.124724Z",
+        "eventName":"ObjectCreated:Put",
+        "userIdentity":{
+            "principalId":"tester"
+        },
+        "requestParameters":{
+            "sourceIPAddress":""
+        },
+        "responseElements":{
+            
"x-amz-request-id":"503a4c37-85eb-47cd-8681-2817e80b4281.5330.903595",
+            "x-amz-id-2":"14d2-zone1-zonegroup1"
+        },
+        "s3":{
+            "s3SchemaVersion":"1.0",
+            "configurationId":"mynotif1",
+            "bucket":{
+                "name":"mybucket1",
+                "ownerIdentity":{
+                    "principalId":"tester"
+                },
+                "arn":"arn:aws:s3:us-east-1::mybucket1",
+                "id":"503a4c37-85eb-47cd-8681-2817e80b4281.5332.38"
+            },
+            "object":{
+                "key":"myimage1.jpg",
+                "size":"1024",
+                "eTag":"37b51d194a7513e45b56f6524f2d51f2",
+                "versionId":"",
+                "sequencer": "F7E6D75DC742D108",
+                "metadata":[],
+                "tags":[]
+            }
+        },
+        "eventId":"",
+        "opaqueData":"[email protected]"
+    }
+]}
+```
+
+As we can see above: there are a number of boilerplate fields to inform us
+of various aspects of the completed operation but there are a few fundamental
+aspects to highlight;
+
+1. the "key" informs us of the key that the operation was performed on.
+
+2. the "eventName" informs us of the type of operation that was
+   performed.  The 2 most notable eventNames are **ObjectCreated:Put** and
+   **ObjectRemoved:Deleted** which pertain to key creation and deletion 
respectively.
+
+3. operation specific fields can be included within the "object" sub-object (in
+   the above example we can see that "size" and "eTag" of the created object 
are included)
+
+## Applicability to Ozone
+
+For non-FSO Ozone buckets / operations there is a clear mapping between
+operations such as CreateKey / CommitKey / DeleteKey / RenameKey and the
+standard S3 event notification semantics.
+
+Examples:
+
+1. CommitKey could be mapped to a ObjectCreated:Put "/path/to/keyToCreate" 
notification event
+
+2. DeleteKey could be mapped to a ObjectRemoved:Deleted "/path/to/keyToDelete" 
notification event
+
+3. RenameKey (assuming a file based key) in standard S3 event noification 
semantics would produce 2 events:
+
+- a ObjectRemoved:Deleted event for the source path of the rename
+- a ObjectCreated:Put event for the destination path of the rename
+
+The challenge in adopting S3 Event notification semantics within Ozone
+would be in at least 2 areas:
+
+### 1. FSO hierarchical operations which impact multiple child keys
+
+Example: directory renames
+
+To illustrate with an example: lets say we have the following simple directory 
structure:
+
+```
+  /vol1/bucket1/myfiles/f1
+  /vol1/bucket1/myfiles/f2
+  /vol1/bucket1/myfiles/subdir/f1
+```
+
+If a user performs a directory rename such as:
+
+```
+  ozone fs -mv /vol1/bucket1/myfiles /vol1/bucket1/myfiles-RENAMED
+```
+
+Within standard S3 event notification semantics we would expect to see 6 
notifications
+emitted in that case:
+
+```
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f1
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f2
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/subdir/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f2
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/subdir/f1
+```
+
+However, with an approach of simply producing notifications based on Ratis
+state machine events then all we would have to go on from the
+RenameKeyRequest would be the fromKeyName and the toKeyName of the
+*parent* of the directory being renamed (and not the impacted child
+objects).

Review Comment:
   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.



##########
hadoop-hdds/docs/content/design/event-notification-schema.md:
##########
@@ -0,0 +1,396 @@
+---
+title: Event notification schema discussion
+summary: Event notifications schema discussion
+date: 2025-06-29
+jira: HDDS-13513
+status: design
+author: Colm Dougan, Donal Magennis
+---
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+## Overview
+
+This document outlines the schema requirements for event notification
+within Ozone and discusses the suitability of 2 widely used event
+notification schemas (S3 and HDFS) as candidates to use as a basis for
+the transmission format for notifications within Ozone.
+
+# General schema requirements
+
+## File/Directory creation/modification
+
+event notifications should be raised to inform consumers of completed
+operations which modify the filesystem and specifically the requests:
+
+#### CreateRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateFileRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+nice to have fields:
+- overwrite
+- recursive
+
+#### CreateDirectoryRequest
+
+we should emit some **create** event
+
+required fields:
+- path (volume + bucket + key)
+- isfile
+
+#### CommitKeyRequest
+
+we should emit some **commit/close** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- data size
+- hsync?
+
+#### DeleteKeyRequest
+
+we should emit some **delete** event
+
+required fields:
+- path (volume + bucket + key)
+
+nice to have fields:
+- recursive (if known)
+
+### RenameKeyRequest
+
+we should emit some **rename** event
+
+required fields:
+- fromPath (volume + bucket + key)
+- toPath (volume + bucket + toKeyName)
+
+nice to have fields:
+- recursive (if known)
+- is directory (if known)
+
+NOTE: in the case of a FSO directory rename there is a dillema
+(discussed later in this document) as to whether we should emit a single
+event for a directory rename (specifying only the old/new directory names)
+or whether we should emit granular events for all the child objects impacted by
+the rename.
+
+## ACLs
+
+event notifications should be raised to inform consumers that ACL events
+have happened. The relevant requests are:
+
+* AddAclRequest
+* SetAclRequest
+* RemoveAclRequest
+
+The fields provided could vary based on the implementation complexity.
+
+Minimally we have a requirement that we be informed that "some ACL update
+happened" to a certain key (or prefix).
+
+Ideally the details would include the full context of the change made as
+per the request. (perhaps by mirroring the full request details as a JSON
+sub-object) e.g. :
+
+```json
+   ...
+
+   "acls": [
+    {
+      type: "GROUP",
+      name: "mygroup"
+      rights: "\000\001",
+      aclScope: "ACCESS",
+    }
+   ]
+```
+
+The precise details we would need to revisit with guidance from the
+community but this is just to set broad brush expectations.
+
+## SetTimes
+
+event notifications should be raised to inform consumers that
+mtime/atime has changed, as per **SetTimesRequest**
+
+# Transmission format
+
+This section discusses 2 widely used transmission formats for event
+notifiations (S3 and HDFS) and their suitability as candidates for
+adoption within Ozone.
+
+It is not assumed that these are the only options available but they are
+good examples to test against our requirements and discuss trade-offs.
+
+## 1. S3 Event Notification schema
+
+The S3 event notification schema:
+
+[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
+
+Notification events are produced as a list of JSON records.
+
+To illustrate we can look at a sample "create" event from the Ceph docs
+(https://docs.ceph.com/en/quincy/radosgw/notifications/#events):
+
+```json
+
+{"Records":[
+    {
+        "eventVersion":"2.1",
+        "eventSource":"ceph:s3",
+        "awsRegion":"us-east-1",
+        "eventTime":"2019-11-22T13:47:35.124724Z",
+        "eventName":"ObjectCreated:Put",
+        "userIdentity":{
+            "principalId":"tester"
+        },
+        "requestParameters":{
+            "sourceIPAddress":""
+        },
+        "responseElements":{
+            
"x-amz-request-id":"503a4c37-85eb-47cd-8681-2817e80b4281.5330.903595",
+            "x-amz-id-2":"14d2-zone1-zonegroup1"
+        },
+        "s3":{
+            "s3SchemaVersion":"1.0",
+            "configurationId":"mynotif1",
+            "bucket":{
+                "name":"mybucket1",
+                "ownerIdentity":{
+                    "principalId":"tester"
+                },
+                "arn":"arn:aws:s3:us-east-1::mybucket1",
+                "id":"503a4c37-85eb-47cd-8681-2817e80b4281.5332.38"
+            },
+            "object":{
+                "key":"myimage1.jpg",
+                "size":"1024",
+                "eTag":"37b51d194a7513e45b56f6524f2d51f2",
+                "versionId":"",
+                "sequencer": "F7E6D75DC742D108",
+                "metadata":[],
+                "tags":[]
+            }
+        },
+        "eventId":"",
+        "opaqueData":"[email protected]"
+    }
+]}
+```
+
+As we can see above: there are a number of boilerplate fields to inform us
+of various aspects of the completed operation but there are a few fundamental
+aspects to highlight;
+
+1. the "key" informs us of the key that the operation was performed on.
+
+2. the "eventName" informs us of the type of operation that was
+   performed.  The 2 most notable eventNames are **ObjectCreated:Put** and
+   **ObjectRemoved:Deleted** which pertain to key creation and deletion 
respectively.
+
+3. operation specific fields can be included within the "object" sub-object (in
+   the above example we can see that "size" and "eTag" of the created object 
are included)
+
+## Applicability to Ozone
+
+For non-FSO Ozone buckets / operations there is a clear mapping between
+operations such as CreateKey / CommitKey / DeleteKey / RenameKey and the
+standard S3 event notification semantics.
+
+Examples:
+
+1. CommitKey could be mapped to a ObjectCreated:Put "/path/to/keyToCreate" 
notification event
+
+2. DeleteKey could be mapped to a ObjectRemoved:Deleted "/path/to/keyToDelete" 
notification event
+
+3. RenameKey (assuming a file based key) in standard S3 event noification 
semantics would produce 2 events:
+
+- a ObjectRemoved:Deleted event for the source path of the rename
+- a ObjectCreated:Put event for the destination path of the rename
+
+The challenge in adopting S3 Event notification semantics within Ozone
+would be in at least 2 areas:
+
+### 1. FSO hierarchical operations which impact multiple child keys
+
+Example: directory renames
+
+To illustrate with an example: lets say we have the following simple directory 
structure:
+
+```
+  /vol1/bucket1/myfiles/f1
+  /vol1/bucket1/myfiles/f2
+  /vol1/bucket1/myfiles/subdir/f1
+```
+
+If a user performs a directory rename such as:
+
+```
+  ozone fs -mv /vol1/bucket1/myfiles /vol1/bucket1/myfiles-RENAMED
+```
+
+Within standard S3 event notification semantics we would expect to see 6 
notifications
+emitted in that case:
+
+```
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f1
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/f2
+  eventName=ObjectRemoved:Deleted, key=/vol1/bucket1/myfiles/subdir/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f1
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/f2
+  eventName=ObjectCreated:Put, key=/vol1/bucket1/myfiles-RENAMED/subdir/f1
+```
+
+However, with an approach of simply producing notifications based on Ratis
+state machine events then all we would have to go on from the
+RenameKeyRequest would be the fromKeyName and the toKeyName of the
+*parent* of the directory being renamed (and not the impacted child
+objects).
+
+Therefore to produce notifications using the standard S3 event
+notification semantics for FSO directory renames we would need to
+consider the trade-offs between compatibility with the normal S3
+semantics for renames vs a custom event type for directory renames.
+
+### most compatible approach
+
+We could introduce some additional processing before emitting notification
+events in the case of a directory rename which "gathers together" (prior
+to the change being committed to the DB) the child objects impacted by
+the directory rename and emits pairs of delete/create events for each
+key (as described above)
+
+Pros:
+- standard S3 event notification rename semantics
+
+Cons:
+- additional processing to pull together the events.  This could mean an
+  unknown amount of additional processing for large directory renames.
+- could be a performance drag if performed on the leader
+
+### custom event type
+
+Conversely - we could opt to not try to be fully compliant with existing S3 
event notification
+semantics since the schema was designed for non-hierarchical filesystems and
+instead create some custom event extension (e.g. ObjectRenamed:) and
+emit just a single event for directory renames which specifies only the parent
+paths impacted by the rename:
+
+e.g.
+```
+  eventName=ObjectReanmed:Reanmed, fromKey=myfiles, toKey=myfiles-RENAMED
+```
+
+.. it would then be up to the notification consumer to deal with the
+different rename event semantics (i.e. that only the parent names were
+notified and not the impacted child objects).
+
+This is the same semantics used in the HDFS inotify directory rename
+event (see below).
+
+Pros:
+- no additional processing when emitting events
+
+Cons:
+- non-standard S3 event notification semantics
+
+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?)

Review Comment:
   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)/F1(O4)
   
   Now within a ratis transaction batch if I have 2 parallel transactions:
   mv d1/d2 -> d1/d5 
   mv d1/d2/d3/F1 -> d1/d2/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.
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to