T116786 introduced MediaWiki #EventBus production [[https://gerrit.wikimedia.org/r/#/admin/projects/mediawiki/extensions/EventBus|via an extension]] utilizing hooks. While adequate for the [[https://phabricator.wikimedia.org/T114443|EventBus MVP]], this is only an interim solution. Ultimately, we need a mechanism that guarantees event delivery (eventual consistency is OK).
The [[ https://wikitech.wikimedia.org/wiki/Event_Platform | Event Platform ]] program extended the work started in T116786 to provide a standardized event producing APIs unified for both production and analytics purposes.
However, in order to build truly reliable new production services with events based on MediaWiki data, we need a single source of truth for MediaWiki data. That source of truth is the MediaWiki MariaDB database. This is only consistently accessible by MediaWiki itself. There is currently no way to consistently expose (real time) MediaWiki state changes to non MediaWiki applications.
We do have events produced by MediaWiki, but these events are decoupled from the MariaDB writes, and there is no guarantee that e.g. every revision table save results in a mediawiki.revision-create event. This means that as of today, MediaWiki events cannot be relied on as a 'source of truth' for MediaWiki data. They are not much more than a best (really good!) effort notification.
Background reading: [[ https://www.confluent.io/blog/turning-the-database-inside-out-with-apache-samza/ | Turning the database inside out ]]
## Why do we need this?
I asked a few stake holders to explain why this is important to them, and they gave me permission to quote them here. These are a few examples of why consistent events are important.
##### WikiData Query Service Updater - T244590
@Zbyszko:
> ... missed events are probably the biggest issue in the system. We have visibility into late and out of order events (and probably mostly buggy events, but there's no way of knowing for sure). Not only that, there are sensible ways of dealing with them, both in general and in our specific situation.
>
> Missed events are, by their nature, invisible to us via standard means and hard to observe in general. Since we also don't really understand the situation when those are dropped, it's hard to assess the impact on WDQS updater. We decided we're ok with it for now, because it's simply still better than the previous solution.
>
> To reiterate - we can deal with lateness and out-of-orderliness - dealing with missed events is order of magnitude a harder challenge.
##### Image Recommendations project - T254768
@gmodena:
> Throughout the month, the state of an article can change. We'll need to track a "revisions events topic" to establish a feedback loop with the
> model re the following state changes (among others):
> 1. Previously unillustrated articles that are now illustrated
> 2. Articles illustrated algorithmically, that have been reverted
> 3. Orthogonal (technically not a MW state change): track which recommendations have been rejected by a client.
>
> Being late in capturing state changes, would result in a degraded UX that will fix itself with time.
> Missing events would be an order of magnitude harder problem to solve.
##### HTML wiki content dumps and other public datasets - T182351
@fkaelin:
> Another category of tools that depend on the correctness of the events are derived datasets that the foundation could publish. This includes the equivalent of the wikidumps on which the analytics wiki history datasets are based, which could be replaced with a snapshot-less and continuous log of revisions. Another example is the html dumps discussed in {T182351}, which the OKAPI team can also relate to, and any number of other datasets that one can think of.
##### [[ https://meta.wikimedia.org/wiki/Wikimedia_Enterprise | Wikimedia Enterprise ]] AKA Okapi
@Protsack.stephan
> if you don't have consistent events, how else would you get the data you need for your use case? - We heavily rely on events to maintain our dataset. Basically we do CDC from event streams to maintain our dataset. Not having consistent events means that our dataset gets out of sync and we need to engineer something on top of events to make sure that it is consistent. Just FYI we are just acknowledging that events may be not consistent and putting that problem into a box for now, but that's probably going to be our next bridge to cross.
## Potential solutions
[[ https://martinfowler.com/eaaDev/EventSourcing.html | Event Sourcing ]] is an approach that event driven architectures use to ensure they have a single consistent source of truth that can be used to build many downstream applications. If we were building an application from scratch, this might be a great way to start. However, MediaWiki + MariaDB already exist as our source of truth, and migrating it to an Event Sourced architecture all at once is intractable.
In lieu of completely re-architecting MediaWiki's data source, there are a few possible approaches to solving this problem in a more incremental way.
---
### Change Data Capture (CDC)
CDC uses the MariaDB replication binlog to produce state change events. This is the same source of data used to keep the read MariaDB replicas up to date.
**Description**
A binlog reader such as [[ https://debezium.io/ | debezium ]] would produce database change events to Kafka. This reader may be able to transform the database change events into a more useful data model (e.g. [[ https://schema.wikimedia.org/repositories/primary/jsonschema/mediawiki/revision/create/latest | mediawiki/revision/create ]]), or transformation maybe done later by a Stream Processing framework such as [[ https://flink.apache.org/ | Flink ]] or [[ https://kafka.apache.org/documentation/streams/ | Kafka Streams ]].
**Pros**
* No MediaWiki code changes needed
* Events are guaranteed to be produced for every database state change
* May be possible to guarantee each event is produced exactly once
* Would allow us to incrementally Event Source MediaWiki (if we wanted to)
**Cons**
* Events are emitted (by default?) in a low level database change model, instead of a higher level domain model, and need to be joined together and transformed by something, most likely a stateful stream processing application.
* WMF's MariaDB replication configuration may not support this (we may need GTIDs).
* Data Persistence is not excited about maintaining more 'unicorn' replication setups.
---
### Transactional Outbox
This makes use of database transactions and a separate poller process to produce events.
See also: https://microservices.io/patterns/data/transactional-outbox.html
**Description**
Here's how this might work:
* MediaWiki wraps MariaDB writes for a web request in one transaction.
* When an event is to be emitted, it is serialized and inserted into an `event_outbox` table.
* Once the web request is finished, MW EventBus attempts to produce the event in a deferred update as it does currently.
* If successful, the previously inserted row in the `event_outbox` table is deleted.
* If failed, the previously inserted row can be updated with a a failed_at timestamp and an error message.
A separate maintenance process polls the `event_outbox` table for rows, produces the events to Kafka, and deletes the row when the produce request succeeds.
NOTE: This example is just one of various ways a Transactional Outbox might be implemented. The core idea is the use of MariaDB transactions and a separate poller to ensure that all events are produced.
**Pros**
* Events can be emitted modeled as we choose
* Since MW generally wraps all DB writes in a transaction, no MW core change needed. This could be done in an extension.
**Cons**
* At least once guarantee for events, but this should be fine. There may be ways to easily detect a the duplicate event.
* Separate polling process to run and manage.
---
### Hybrid: Change Data Capture via Transactional Outbox
This is a hybrid of the above two approaches. The main difference is instead of using CDC to emit change events on all MariaDB tables, we only emit change events for event outbox tables.
This idea is from Debezium: https://debezium.io/blog/2019/02/19/reliable-microservices-data-exchange-with-the-outbox-pattern/
**Description**
MediaWiki would be configured to write all changes in a transaction with the outbox tables. When a revision is to be inserted into the `revision` table, a MariaDB transaction is started. A record is inserted into the `revision` table as well as the `revision_event_outbox` table. The `revision_event_outbox` has a field including a JSON string representing the payload of the change event. The transaction is then committed.
A binlog reader such as Debezium would then filter for changes to the `revsion_event_outbox` table (likely extracting only the JSON event payload) and emit only those to Kafka.
**Pros**
* Events can be emitted modeled as we choose
* Events are guaranteed to be produced for every database state change
* May be possible to guarantee each event is produced exactly once
* No need to transform from low level database changes to high level domain models.
* Since MW generally wraps all DB writes in a transaction, no MW core change needed. This could be done in an extension.
* Would allow us to incrementally Event Source MediaWiki (if we wanted to)
**Cons**
* WMF's MariaDB replication configuration may not support this (we may need GTIDs).
* Data Persistence is not excited about maintaining more 'unicorn' replication setups.
---
### Reconciliation Capability
> Devise a way to be able to link an event to the database transaction/revid/whatever that generated it, and have a system that allows reconciliation. For instance: one revision is missing from the event log but is present in the database? then generate the event that we're missing.
e.g. {T358373}
---
### 2 Phase Commit with Kafka Transactions
This may or may not be possible and requires more research if we want to consider it. Implementing it would likely be difficult and error prone, and could have an adverse affect on MediaWiki performance. If we do need Kafka Transactions, this might be impossible anyway, unless a good PHP Kafka Client is written.