Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
396 changes: 396 additions & 0 deletions hadoop-hdds/docs/content/design/event-notification-schema.md
Original file line number Diff line number Diff line change
@@ -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)
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.

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


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)

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'

### 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**
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.


# 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
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?


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",

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"?

"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?

"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).
Comment on lines +280 to +282
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.


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


### 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
Loading