Modelling Exactly-Once Using TLA+ Part 3: Modelling Failures and Transactions

Farooq Qaiser
21 min readSep 10, 2024

--

TLA+ is a language for writing system specifications and discovering fundamental design errors quickly. In this series of blogposts, we’ll introduce TLA+ by iteratively modelling and refining our design for a simple application that can read and write messages from/to Kafka in an exactly-once fashion. This will be a four-part series consisting of:

  1. Writing a Specification
  2. Checking a Specification
  3. Modelling Failures and Transactions
  4. Modelling Zombies and Zombie Fencing (coming soon)

You can find all of the code for this series in this GitHub repo.

Also, if you have interesting systems engineering problems and are looking for a strong Software Engineer, feel free to reach out via LinkedIn.

Introduction

We ended the last post on a bit of a cliffhanger suggesting our specification is not useful because it makes some unrealistic assumptions. In this post, we’ll start by trying to uncover these assumptions by generating state graphs. After that, we’ll refactor our spec to remove these assumptions and make it more realistic. However, once we do that, we’ll discover problems in the exactly-once-ness of our design. And so, we’ll finish by updating our design to make it robust to duplicates. Buckle up, after all of our hard work in parts 1 and 2, this is where things finally start to get interesting!

State graph

As already mentioned, our specification currently has some unrealistic assumptions embedded within it. Unfortunately, as TLA+ newbies, it might be hard to see these assumptions from just reading the spec. To get a slightly different perspective, let’s try to visualize our spec as a state graph.

State graphs are a fairly intuitive way of visualizing the behaviours of a system, where nodes (boxes) are used to represent distinct states and edges (arrows) are used to represent possible state transitions. Since TLA+ specs are already a formal description of the behaviours of a system, it should come as no surprise that we can generate a state graph directly from our spec using TLC and Graphviz (installation instructions here).

The first step is to translate our spec into a DOT file using TLC:

java -cp tla2tools.jar tlc2.TLC -dump dot,colorize,actionLabels ExactlyOnce.dot ExactlyOnce.tla

We can then generate a visualization from the DOT file using Graphviz:

dot -Tpng ExactlyOnce.dot -o ExactlyOnce.png

Which should generate an image similar to this:

Figure 1

This graph shows all the states and state transitions that are possible in our system as we’ve specified it. We have two boxes indicating the two possible states our system can be in at any moment in time. We also have arrows showing all the possible steps our system can take. The solid arrow indicates the only possible way to go from our initial state to our terminal state is via the ConsumeProduceCommit step. We also have a pair of dashed arrows to indicate steps where no variables change i.e. stuttering steps and/or Done steps.

As far as systems go, this is an incredibly simple one. In fact, if our system was truly this simple, it would not be worth our time modelling it in TLA+. However before we prematurely conclude this series has been a waste of time, let’s consider what assumptions are embedded inside our ConsumeProduceCommit step.

Atomicity

If we compare the state of our system before and after the ConsumeProduceCommit step, we can see we’re assuming that our system will during this single step:

  1. Successfully consume a message
  2. And successfully produce a message
  3. And successfully commit our consumer offset

In other words, we’re assuming that we can build a system that can consume, produce, and commit messages from/to Kafka as a single atomic operation i.e. either the operation as a whole is executed completely or it is not executed at all but it is never executed partially.

Unfortunately, it’s impossible to consume, produce, and commit messages from/to Kafka as a single atomic operation in the real world. Each of consume, produce, and commit are independent operations that can potentially fail and as a result, there is no way to guarantee we won’t fail partway through the process e.g. we could fail after consuming and end up not producing or committing the consumed message.

This is a gentle reminder that just because we can express something in TLA+ is no guarantee that we can actually build such a thing in real life. In fact, given how flexible the TLA+ language is, there are far more things we can express in TLA+ than we can actually build.

As practical engineers, it’s not particularly useful to write specifications for systems we know we can’t build. Over the next couple of sections, we’ll gradually refactor our specification so it better reflects the realities of programming.

Small Steps

In TLA+, a step is atomic. TLC will not execute a step partially. Either all of the variable changes defined by a step will happen or none at all. As we already noted, we cannot consume, produce, and commit as a single atomic operation in the real world so it doesn’t make sense to model these as a single step in our TLA+ spec. We need to break up our “big” ConsumeProduceCommit step into smaller steps that better reflect the atomicity guarantees we have in the real world i.e. we need to model Consume, Produce, and Commit as separate steps in our spec. A naive first attempt at modelling this might look like the following:

Consume ==
/\ MessagesAvailable
/\ Buffer' = InputTopic[CommittedConsumerOffset]
/\ UNCHANGED <<InputTopic, OutputTopic, CommittedConsumerOffset>>

Produce ==
/\ OutputTopic' = Append(OutputTopic, Buffer)
/\ UNCHANGED <<InputTopic, Buffer, CommittedConsumerOffset>>

Commit ==
/\ CommittedConsumerOffset' = Buffer + 1
/\ UNCHANGED <<InputTopic, Buffer, OutputTopic>>

Next ==
\/ Consume
\/ Produce
\/ Commit
\/ Done

While this is certainly a step in the right direction, it still isn’t quite correct. Before you continue reading, take a moment to see if you can spot the problem.

Modelling Sequential Logic

We can find the problem relatively easily by running TLC which should throw the following RuntimeException:

Error: TLC threw an unexpected exception.
This was probably caused by an error in the spec or model.
See the User Output or TLC Console for clues to what happened.
The exception was a java.lang.RuntimeException: Attempted to apply the operator overridden by the Java method
public static tlc2.value.impl.IntValue tlc2.module.Integers.Plus(tlc2.value.impl.IntValue,tlc2.value.impl.IntValue),
but it produced the following error:
Cannot cast tlc2.value.impl.ModelValue to tlc2.value.impl.IntValue

Error: The behavior up to this point is:
State 1: <Initial predicate>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<>>
/\ Buffer = NULL
/\ InputTopic = <<1>>

Error: The error occurred when TLC was evaluating the nested
expressions at the following positions:
0. Line 29, column 5 to line 30, column 52 in ExactlyOnce
1. Line 29, column 8 to line 29, column 44 in ExactlyOnce
2. Line 29, column 35 to line 29, column 44 in ExactlyOnce

From the error message, we can see the RuntimeException is thrown inside our Commit step (line 29 refers to our Commit definition) when TLC attempts to apply the + operator while trying to compute the next CommittedConsumerOffset value based on the current Buffer value. Unfortunately, Buffer is equal to NULL and since we’ve modelled NULL as a model value and model values don’t support any operation other than equality, we get a RuntimeException.

That’s the problem at the code level. From a modelling perspective though, the real question we need to ask ourselves is why is Buffer equal to NULL when we reach the Commit step? Or rather, why does Buffer not contain a message like it’s supposed to do by the time we reach the Commit step? The answer to that can also be found in our error message. TLC shows us the sequence of states it explored before hitting this error. There is only one state; the initial state. This means that TLC attempted to take Commit as the very first step in our system! This is obviously wrong; we can’t commit a message before we’ve even consumed it.

So now we need to understand why TLC is taking Commit as the first step. If you look closely at our Commit definition, you will note that our spec doesn’t specify any conditions for when it is appropriate to take aCommit step. In other words, our spec says we can take a Commit step at any time. In fact, our Produce step also suffers from the same problem; it can also be taken at any moment in time!

What we need is a way to say that Consume, Produce, and Commit steps need to happen sequentially. This is easy to do in regular programming languages where we have explicit control flow and is probably something you normally take for granted. However, as we’ve said before, TLA+ is not a programming language. If anything, TLA+ is a language for defining state machines and if we want to allow only certain state transitions, we have to specify them explicitly. In order to do that, we’ll need to introduce the concept of control flow into our specification:

VARIABLES InputTopic, Buffer, OutputTopic, CommittedConsumerOffset, PC    \* 1

vars == <<InputTopic, Buffer, OutputTopic, CommittedConsumerOffset, PC>>
\* 2

Init ==
/\ InputTopic = <<1>>
/\ Buffer = NULL
/\ OutputTopic = <<>>
/\ CommittedConsumerOffset = 1
/\ PC = "Consume"
\* 3

Consume ==
/\ PC = "Consume"
\* 4
/\ IF MessagesAvailable
THEN /\ Buffer' = InputTopic[CommittedConsumerOffset]
/\ PC' = "Produce"
\* 5
ELSE /\ Buffer' = NULL
/\ PC' = "Done"
\* 6
/\ UNCHANGED <<InputTopic, OutputTopic, CommittedConsumerOffset>>

Produce ==
/\ PC = "Produce"
\* 7
/\ OutputTopic' = Append(OutputTopic, Buffer)
/\ PC' = "Commit"
\* 8
/\ UNCHANGED <<InputTopic, Buffer, CommittedConsumerOffset>>

Commit ==
/\ PC = "Commit"
\* 9
/\ CommittedConsumerOffset' = Buffer + 1
/\ PC' = "Consume"
\* 10
/\ UNCHANGED <<InputTopic, Buffer, OutputTopic>>

Done ==
/\ PC = "Done"
\* 11
/\ UNCHANGED vars
  1. We’ll use the same technique PlusCal (a language that compiles down to TLA+) does for modelling control flow and introduce a new variable PC to explicitly represent the program counter state in our spec. The term “program counter” is usually used to refer to a special register in a computer’s processor that contains the memory address for the next instruction to be executed. For the purposes of our specification though, we can model this more simply as a string value indicating the next step to be executed.
  2. Just like every other variable in our system, we need to make sure to add PC to our vars sequence.
  3. Our initial state predicate just needs to specify the initial value of the PC variable, which we’ll say should be "Consume".
  4. In our Consume step, we add a new condition that specifies that we can only take this step as long as PC = "Consume". This change and the change to our initial state predicate are what ensure the first step our system can take is only ever theConsume step.
  5. In addition, each step now needs to specify what the next value of PC should be. So in our Consume step, in the case where there are still messages available, we specify the value of PC in the next state should be "Produce".
  6. In the case where there are no more messages available, we specify the value of PC in the next state should be "Done".
  7. Similarly to what we did in our Consume step, in our Produce step we add a new condition that specifies that we can only take this step as long as PC = "Produce".
  8. We also specify the value of PC in the next state should be "Commit".
  9. In our Commit step, we add a new condition that specifies that we can only take this step as long as PC = "Commit".
  10. We also specify the value of PC in the next state should be "Consume".
  11. Lastly, we specify in our Done step that we can only take this step as long as PC = "Done".

We can verify visually if we’ve defined our state machine correctly by generating another state graph:

Figure 2

That’s looking more like how we expect our program to flow.

However, having 3 small steps that always succeed is not that much different from having 1 big step that always succeeds. Indeed, if we run TLC, we should see our spec still satisfies all of our desired properties and invariants. In reality though, there is no guarantee that our process won’t fail in between steps. This could be because of a random power surge, a network blip, nuclear fallout, an alien wielding a laser gun etc.

Image generated using Microsoft Designer with the prompt “cartoon alien wielding laser gun”

In order to make our model more realistic, we need to introduce the possibility of random failures in between steps.

Modelling Failures

As distributed systems engineers will happily remind you, things can fail anywhere and at any time. Our process could fail immediately. Our process could fail after consuming a message. Our process could fail after producing a message. Our process could fail after committing offsets. Each time our process fails, it must start all over again from the Consume step. We can define this behaviour as yet another step in our spec:

Restart ==                                                             \* 1
/\ PC /= "Done"
\* 2
/\ Buffer' = NULL
\* 3
/\ PC' = "Consume"
\* 4
/\ UNCHANGED <<InputTopic, OutputTopic, CommittedConsumerOffset>>
\* 5

Next ==
\/ Consume
\/ Produce
\/ Commit
\/ Done
\/ Restart
\* 6
  1. We’ll call the operator defining this step Restart. The idea here is that this step always restarts our process from the Consume step.
  2. We’ll specify that we can only take this step as long as PC does not equal to "Done". The reason for this is that our Done step really represents a system that has terminated so it doesn’t make a whole lot of sense for our our application to be able to restart if it’s already terminated.

    If you’re wondering why we don’t have a PC = "Restart" expression in our Restart step like we do for other steps, it’s important to remember that PC represents our program counter state and failures/restarts aren’t really a program counter state.
  3. Here, we’re simulating the loss of any in-memory state when we restart by specifying the next value of Buffer as NULL.
  4. We specify that our process has to start from the Consume step all over again by specifying the value of PC in the next state as "Consume".
  5. As usual, we have to specify all the other variables that are left unchanged by this step.
  6. Finally, we add Restart as yet another option to our Next disjunction.

That’s pretty much all we need to do to simulate random failures. At this point, we can try running our spec through TLC to see if we’re still able to maintain our desired properties and invariants in the presence of random failures. Unfortunately, you should see our NoDuplicates invariant is now being violated in at least one sequence of steps:

Error: Invariant NoDuplicates is violated.

Error: The behavior up to this point is:

State 1: <Initial predicate>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<>>
/\ Buffer = NULL
/\ InputTopic = <<1>>
/\ PC = "Consume"

State 2: <Consume line 21, col 5 to line 27, col 69 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<>>
/\ Buffer = 1
/\ InputTopic = <<1>>
/\ PC = "Produce"

State 3: <Produce line 30, col 5 to line 33, col 64 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<1>>
/\ Buffer = 1
/\ InputTopic = <<1>>
/\ PC = "Commit"

State 4: <Restart line 46, col 5 to line 49, col 69 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<1>>
/\ Buffer = NULL
/\ InputTopic = <<1>>
/\ PC = "Consume"

State 5: <Consume line 21, col 5 to line 27, col 69 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<1>>
/\ Buffer = 1
/\ InputTopic = <<1>>
/\ PC = "Produce"

State 6: <Produce line 30, col 5 to line 33, col 64 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<1, 1>>
/\ Buffer = 1
/\ InputTopic = <<1>>
/\ PC = "Commit"

Notice how OutputTopic is equal to <<1, 1>> in the final state; these are the duplicates that are violating our NoDuplicates invariant. We can also generate a state graph to see the problem more visually:

Figure 4

Note that this visualization doesn’t show the full state space, just the state space that was explored before TLC discovered a state where our NoDuplicates property was violated, which you can see in the bottom-right corner.

You could try to explore the whole state space by removing the NoDuplicates invariant temporarily from your TLC configuration file. However, in that case, you would find that TLC never finishes exploring as there are an infinite number of states that TLC will explore without the NoDuplicates invariant. You could have 1 duplicate, 2 duplicates, 3 duplicates, etc.

Discovering problems like this can be difficult in regular programming because it’s usually hard to simulate failures at exactly the right point in our code. With TLA+ however it’s easy as TLC will always attempt to fully explore the state space.

So, we made our model more realistic and subsequently discovered a flaw in our design. Now we need to fix it. In the next section, we’ll discuss how we can adapt our design so that we can continue to maintain our NoDuplicates invariant even in a world with random failures.

Random failures might seem like a edge case scenario but this in only true in the small. Past a certain point of scale, problems that were previously only rare occurrences start to become likely if not common events (and don’t forget about correlated/compounding failures!). What this means is that, if you plan to scale, failures are a reality that you need to account for in your design. Also, if you’re going to call something exactly-once, it had better be exactly-once not just in the happy paths but also when things are going wrong.

Transactions

Before we jump into solution-ing, let’s try to understand the problem a little better. If we take a closer look at our state graph (Figure 4), we can see that not all Restart steps result in duplicates. For example, following the sequence of steps highlighted in green below does not result in duplicates in our OutputTopic despite the fact that this sequence allows for restarts in 4 different places.

Figure 5

On the other hand, if our system follows the sequence of steps highlighted in red below, we do end up with duplicates in our OutputTopic.

Figure 6

Based on this, we can make a few specific observations:

  • If we fail after our process launches, this does not result in duplicates
  • If we fail after our Consume step, this does not result in duplicates
  • If we fail after our Produce step, this does result in duplicates
  • If we fail after our Commit step, this does not result in duplicates

This should make intuitive sense! Once we’ve produced a message, it’s gone. If we fail to commit our consumer offset, we’ll restart from the last committed consumer offsets and end up consuming and producing the same message all over again. Failing at any other point in our code has no impact on duplicates.

Therefore, we could argue that our fundamental problem is that our system does not guarantee that we will always commit our offsets after we’ve produced a message. This is the “critical phase” in our program logic. If we could somehow eliminate the possibility of failure between producing and committing a message, we should be able to close this source of duplicates. Luckily, we don’t have to think about this too hard in our heads; we can just test out our theory quickly using TLA+!

ProduceCommit ==                                   \* 1
/\ PC = "ProduceCommit"
/\ OutputTopic' = Append(OutputTopic, Buffer)
/\ CommittedConsumerOffset' = Buffer + 1
/\ PC' = "Consume"
/\ UNCHANGED <<InputTopic, Buffer>>

Next ==
\/ Consume
\/ ProduceCommit
\* 2
\/ Done
\/ Restart
  1. Here we’re just combining our Produce and Commit steps back together. Since a step is atomic in TLA+, there’s no way we can fail between producing and committing now.
  2. Here we’ve replaced our previously separate Produce and Commit steps with our new atomicProduceCommit step inside our Next disjunction.

This probably feels like we’re taking a step backwards. Didn’t we just split Consume, Produce, and Commit into separate steps because it was impossible to build a system that could execute all three actions as a single atomic operation? The difference here is that it is possible to build a system where we can Produce and Commit messages to Kafka as a single atomic operation using the Kafka Transactions API that was added in version 0.11.

In reality, even if you’re using the Kafka Transactions API, you will still in most cases still end up producing messages to your topic before you commit the transaction. However, your consumers will not be able to read those messages until the transaction is committed (assuming you’ve correctly configured your consumers with isolation.level = "read_committed"). So the overall, observable effect is that of an all-or-nothing operation from the consumer’s perspective. For this reason, it’s correct to model this API as a single atomic step. To learn more about how transactions are implemented in Kafka, see KIP-98.

So we might have a solution that we can actually implement in reality! We just need to make sure this new design actually satisfies our desired properties. Running TLC should reveal that we’re no longer violating our NoDuplicates invariant, hooray! Unfortunately, we’re violating a different property now; Termination.

Strong Fairness

TLC is reporting that our Termination property is being violated in the following sequence of events:

Error: Temporal properties were violated.

Error: The following behavior constitutes a counter-example:

State 1: <Initial predicate>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<>>
/\ Buffer = NULL
/\ InputTopic = <<1>>
/\ PC = "Consume"

State 2: <Consume line 21, col 5 to line 27, col 69 of module ExactlyOnce>
/\ CommittedConsumerOffset = 1
/\ OutputTopic = <<>>
/\ Buffer = 1
/\ InputTopic = <<1>>
/\ PC = "ProduceCommit"

Back to state 1: <Restart line 41, col 5 to line 45, col 69 of module ExactlyOnce>

Looking carefully, we can see that our ProduceCommit step is never taken in this sequence of steps. Our system always restarts before it can take this step, returning the system back to the initial state in a never-ending loop. As a result, our consumer offset never advances and we therefore never satisfy our definition of Termination.

While it’s nice to confirm that our system will never terminate if we always restart before committing our offsets, it’s much more useful to assume that even if the system restarts a few times, eventually the system will take a ProduceCommit step. If this is sounding familiar, that’s because we faced a similar problem in Part 2 with infinite stuttering steps. We fixed the problem there by introducing a weak fairness constraint. We can do something similar here as well:

Fairness ==
/\ WF_vars(Next)
/\ SF_vars(ProduceCommit)
\* 1
  1. Previously, we defined Fairness as simply WF_vars(Next) i.e. weak fairness of the Next action. Weak fairness says that if an action is always enabled, it eventually happens. Our problem is that the ProduceCommit step isn’t always enabled. It is only intermittently enabled. Weak fairness does not guarantee that intermittently enabled actions will eventually happen. For those kind of semantics, we need to specify our action as strongly fair. We say an action is strongly fair if, given that said action isn’t permanently disabled, it will eventually happen.

    Similar to how we declare a step as weakly fair, we can mark a step as strongly fair using the SF_v(A) syntax, where A is the name of the action/step that changes variable v.

If we rerun TLC now, we should see it report that our spec satisfies all of our declared properties and invariants now. We can confirm the same visually as well by generating a state graph from our spec:

Figure 7

That’s looking much “cleaner” now.

Alternatively, instead of making our ProduceCommit step strongly fair, we could remove Termination and NoMissingData as desired properties. That’s a rational design decision to make in a world where avoiding duplicates is an invariant we want to uphold at all costs, even if that means our process stops making progress.

Spec

After all of our changes, this is what our TLA+ spec looks like now:

---- MODULE ExactlyOnce ----

EXTENDS Sequences, Integers

CONSTANT NULL

VARIABLES InputTopic, Buffer, OutputTopic, CommittedConsumerOffset, PC

vars == <<InputTopic, Buffer, OutputTopic, CommittedConsumerOffset, PC>>

Init ==
/\ InputTopic = <<1>>
/\ Buffer = NULL
/\ OutputTopic = <<>>
/\ CommittedConsumerOffset = 1
/\ PC = "Consume"

MessagesAvailable == CommittedConsumerOffset < Len(InputTopic) + 1

Consume ==
/\ PC = "Consume"
/\ IF MessagesAvailable
THEN /\ Buffer' = InputTopic[CommittedConsumerOffset]
/\ PC' = "ProduceCommit"
ELSE /\ Buffer' = NULL
/\ PC' = "Done"
/\ UNCHANGED <<InputTopic, OutputTopic, CommittedConsumerOffset>>

ProduceCommit ==
/\ PC = "ProduceCommit"
/\ OutputTopic' = Append(OutputTopic, Buffer)
/\ CommittedConsumerOffset' = Buffer + 1
/\ PC' = "Consume"
/\ UNCHANGED <<InputTopic, Buffer>>

Done ==
/\ PC = "Done"
/\ UNCHANGED vars

Restart ==
/\ PC /= "Done"
/\ Buffer' = NULL
/\ PC' = "Consume"
/\ UNCHANGED <<InputTopic, OutputTopic, CommittedConsumerOffset>>

Next ==
\/ Consume
\/ ProduceCommit
\/ Done
\/ Restart

Fairness ==
/\ WF_vars(Next)
/\ SF_vars(ProduceCommit)

Spec ==
/\ Init
/\ [][Next]_vars
/\ Fairness

Termination == <>[](~MessagesAvailable)

NoDuplicatesIn(seq) ==
\A i, j \in 1..Len(seq):
i /= j => seq[i] /= seq[j]

NoDuplicates == NoDuplicatesIn(OutputTopic)

SeqContains(seq, value) ==
\E x \in 1..Len(seq):
seq[x] = value

SeqContainsAll(seq, values) ==
\A i \in 1..Len(values):
SeqContains(seq, values[i])

NoMissingData == <>[](SeqContainsAll(OutputTopic, InputTopic))

====

We didn’t make any changes to our TLC file but for completeness’ sake, here it is:

SPECIFICATION Spec
CONSTANT NULL = NULL
INVARIANT NoDuplicates
PROPERTY Termination NoMissingData

Reflections

A good chunk of the work we do when writing TLA+ is just abstraction. The challenge here is finding the right level of abstraction. We don’t want to abstract too much or we risk losing important details/nuances of our problem/domain. This was arguably part of the problem we had at the beginning of this post; we’d abstracted away too much of our program logic when we had a single ConsumeProduceCommit step and consequently missed important duplicate-generating edge cases.

At the same time, we don’t want to abstract too little either. For example, we chose to model Consume, Produce, and Commit as fairly high level steps. We could have gone much further and modelled all 488 pages of the Kafka protocol however this would require considerable effort for very little gain.

Finding a good balance is more of an art than a science and will depend heavily on your use case. That said, correct abstraction requires a solid knowledge of your domain. There’s no getting around that. For example, it would be dangerous to assume that we can model Kafka transactions as single ProduceCommit step without a good understanding of the relevant APIs. In addition, it’s always a good idea to critically review the “grain of atomicity” at which you’re modelling operations in TLA+ to make sure they’re realistic. Lastly, while it’s probably more common for users to abstract too much rather than too little, it’s helpful to remember that TLA+ specifications are “seldom more than about 2000 lines.” So if your specifications are starting to reach that range, you might just be abstracting too little.

However, once we got our model to the right level of abstraction (by modelling separate steps for Consume, Produce, and Commit, and modelling the possibility of arbitrary failures), TLC was easily able to surface issues in the exactly-once-ness of our design and we were able to quickly iterate from there with TLC acting almost like our copilot.

So are we done now? Surely after nearly 15,000 words across 3 blogposts, we must have a design that is truly exactly-once by now? 🙃 So far we’ve assumed that there is only ever one process that is consuming from the input topic and writing to the output topic. What happens when we have multiple processes running? Is it possible to generate duplicates in such a scenario? Tune in next time for an epic finale involving zombies and the noble art of fencing.

Image generated using Microsoft Designer with the prompt “Fencer versus a zombie caroon style”

An Important Reminder

Since part 2 of this series:

  1. By now, you must have heard the news about how a 26 year old American peace activist, Ayşenur Ezgi Eygi, was killed by an Israeli occupation sniper’s bullet to her head. If not, I encourage you to read this beautiful tribute written by her friend to learn more about the kind of person she was and the disturbing circumstances under which she was killed.
  2. In this brief but harrowing interview, Australian doctor Mohammad Mustafa provides a first-hand account of the state of the practically non-existent healthcare system remaining in Gaza. This is far from the first time doctors have spoken out. In July, 45 healthcare professionals who bravely chose to volunteer in Gaza hospitals put together an 8 page letter addressed to Biden and others, which I encourage you to read in whole. The following quotes are from the same letter:

    “With only marginal exceptions,
    everyone in Gaza is sick, injured, or both.”

    “… every single signatory to this letter treated children in Gaza who suffered violence that must have been deliberately directed at them. Specifically, every one of us on a daily basis treated pre-teen children who were shot in the head.”

    “… Israel has directly targeted and deliberately devastated Gaza’s entire healthcare system, and that Israel has targeted our colleagues in Gaza for death, disappearance, and torture.”
  3. In addition, Israeli troops stand accused of, repeatedly, firing at humanitarian workers, targeting journalists, using Palestinian civilians as human shields, raping prisoners, bombing schools, and much more.

Don’t be a bystander to genocide.

🇵🇸🇵🇸🇵🇸

--

--