diff --git a/hadoop-hdds/docs/content/design/event-notification-schema.md b/hadoop-hdds/docs/content/design/event-notification-schema.md
new file mode 100644
index 000000000000..e10ac7ee3960
--- /dev/null
+++ b/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
+---
+
+
+## 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":"me@example.com"
+ }
+]}
+```
+
+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?)
+
+### 2. Metadata changes
+
+The standard S3 event notification schema does not have provision for
+notifying about metadata changes.
+
+Therefore to support notifying about metadata changes one option would
+be to add a custom event type. e.g. ObjectMetadataUpdated:*
+
+It is worth noting here that Ceph has some custom extensions,
+so there is some precedent for that:
+https://docs.ceph.com/en/latest/radosgw/s3-notification-compatibility/#event-types
+
+
+## 2. HDFS event schema
+
+The HDFS inotify event notification schema
+
+[https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/hdfs/inotify/package-summary.html](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/hdfs/inotify/package-summary.html)
+
+allows a HDFS client with suitable privileges to poll the HDFS namenode
+for notifications pertaining to changes on the filesystem across the entire cluster
+(i.e. there is no granular per-directory subscription).
+
+The notifications use a binary protocol (protobuf). The protobuf specs
+for the notification events can be found here:
+
+https://github.com/apache/hadoop/blob/3d905f9cd07d118f5ea0c8485170f5ebefb84089/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/inotify.proto#L62
+
+
+## Applicability to Ozone
+
+Since HDFS is a hierarchical filesystem there is a natural mapping to
+the FSO operations within Ozone.
+
+For example:
+
+* a directory rename is emitted as a RenameEvent
+ (https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/hdfs/inotify/Event.RenameEvent.html) with
+ srcPath=/path/to/old-dir, dstPath=/path/to/new-dir (i.e. there is no
+ expectation that the impact on child objects will be notified)
+
+* a recursive delete is emitted as a UnlinkEvent
+ (https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/hdfs/inotify/Event.UnlinkEvent.html) on the parent
+
+* metadata changes (such as changes to permissions, replication,
+ owner/group, acls, xattr etc.
+
+are sent via a MetadataUpdateEvent
+(https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/hdfs/inotify/Event.MetadataUpdateEvent.html)
+
+This would be a good starting point for Ozone but would require some
+bespoke changes as acls, for example, do not have a one-to-one mapping
+to HDFS concepts.
+
+Pros:
+- clear mapping for FSO and non-FSO operations such as directory renames
+- caters for metadata operations by design (although would require some
+ customization)
+
+Cons:
+- not ubiquitous across many storage solutions in the way that the S3 Event Notification schema is
diff --git a/hadoop-hdds/docs/content/design/event-notifications.md b/hadoop-hdds/docs/content/design/event-notifications.md
new file mode 100644
index 000000000000..aa5042790e4e
--- /dev/null
+++ b/hadoop-hdds/docs/content/design/event-notifications.md
@@ -0,0 +1,255 @@
+---
+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
+---
+
+
+# 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. Use DeleteRange API for cleaning up events outside of the retention policy.
+
+#### 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.
+Reads are done asynchronously by plugins in their own threads.
+
+## Configuration
+
+Configuration has two parts. Firstly the configuration of the persistence events, secondly the configuration of the plugins.
+
+e.g. to enable notification persistence
+```xml
+
+
+ ozone.event.notification.enabled
+ true
+
+
+ ozone.event.notification.limit
+ 1000000
+
+
+```
+Example configuration to provide a kafka plugin:
+```xml
+
+
+
+ ozone.om.plugin.kafka
+ true
+
+
+ ozone.om.plugin.kafka.class
+ org.apache.ozone.notify.KafkaPublisher
+
+
+ ozone.om.plugin.kafka.bootstrap.server
+
+
+
+ ozone.om.plugin.kafka.topic
+
+
+
+ ozone.om.plugin.kafka.saslUsername
+
+
+
+ ozone.om.plugin.kafka.saslPassword
+
+
+
+ ozone.om.plugin.kafka.saslMechanism
+
+
+
+ ozone.om.plugin.kafka.clientTlsCert
+
+
+
+ ozone.om.plugin.kafka.clientTlsKey
+
+
+
+```
+
+## Metrics
+
+Key metrics would include:
+
+Total events written
+Head TxnId
+Tail TxnId
+Plugin current txnId
+Missed total
+
+## Notification Schema
+
+[event-notification-schema.md](event-notification-schema.md)
+
+## Testing
+
+Unit testing of individual components.
+
+Integration tests with Kafka/RabbitMQ.