use exclusive fromSequenceNumber in eventsByTag, #22145

* The reason is to have a consistent approach for Sequence and
  TimeBasedUUID, which are both intended as unique event identifiers.
* This means that you can use the offset that is returned in `EventEnvelope`
  as the `offset` parameter in a subsequent query.
This commit is contained in:
Patrik Nordwall 2017-01-16 11:13:45 +01:00
parent 4a9c753710
commit 8083c0bf4a
15 changed files with 101 additions and 34 deletions

View file

@ -17,10 +17,28 @@ object Offset {
abstract class Offset
/**
* Corresponds to an ordered sequence number for the events. Note that the corresponding
* offset of each event is provided in the [[akka.persistence.query.EventEnvelope]],
* which makes it possible to resume the stream at a later point from a given offset.
*
* The `offset` is exclusive, i.e. the event with the exact same sequence number will not be included
* in the returned stream. This means that you can use the offset that is returned in `EventEnvelope`
* as the `offset` parameter in a subsequent query.
*/
final case class Sequence(value: Long) extends Offset with Ordered[Sequence] {
override def compare(that: Sequence): Int = value.compare(that.value)
}
/**
* Corresponds to an ordered unique identifier of the events. Note that the corresponding
* offset of each event is provided in the [[akka.persistence.query.EventEnvelope]],
* which makes it possible to resume the stream at a later point from a given offset.
*
* The `offset` is exclusive, i.e. the event with the exact same sequence number will not be included
* in the returned stream. This means that you can use the offset that is returned in `EventEnvelope`
* as the `offset` parameter in a subsequent query.
*/
final case class TimeBasedUUID(value: UUID) extends Offset with Ordered[TimeBasedUUID] {
if (value == null || value.version != 1) {
throw new IllegalArgumentException("UUID " + value + " is not a time-based UUID")
@ -29,6 +47,9 @@ final case class TimeBasedUUID(value: UUID) extends Offset with Ordered[TimeBase
override def compare(other: TimeBasedUUID): Int = value.compareTo(other.value)
}
/**
* Used when retrieving all events.
*/
final case object NoOffset extends Offset {
/**
* Java API:

View file

@ -78,7 +78,7 @@ private[akka] abstract class AbstractEventsByTagPublisher(
def receiveIdleRequest(): Unit
def timeForReplay: Boolean =
(buf.isEmpty || buf.size <= maxBufSize / 2) && (currOffset <= toOffset)
(buf.isEmpty || buf.size <= maxBufSize / 2) && (currOffset < toOffset)
def replay(): Unit = {
val limit = maxBufSize - buf.size
@ -94,7 +94,7 @@ private[akka] abstract class AbstractEventsByTagPublisher(
persistenceId = p.persistenceId,
sequenceNr = p.sequenceNr,
event = p.payload)
currOffset = offset + 1
currOffset = offset
deliverBuf()
case RecoverySuccess(highestSeqNr)
@ -143,13 +143,13 @@ private[akka] class LiveEventsByTagPublisher(
override def receiveIdleRequest(): Unit = {
deliverBuf()
if (buf.isEmpty && currOffset > toOffset)
if (buf.isEmpty && currOffset >= toOffset)
onCompleteThenStop()
}
override def receiveRecoverySuccess(highestSeqNr: Long): Unit = {
deliverBuf()
if (buf.isEmpty && currOffset > toOffset)
if (buf.isEmpty && currOffset >= toOffset)
onCompleteThenStop()
context.become(idle)
}
@ -174,7 +174,7 @@ private[akka] class CurrentEventsByTagPublisher(
override def receiveIdleRequest(): Unit = {
deliverBuf()
if (buf.isEmpty && currOffset > toOffset)
if (buf.isEmpty && currOffset >= toOffset)
onCompleteThenStop()
else
self ! Continue
@ -184,7 +184,7 @@ private[akka] class CurrentEventsByTagPublisher(
deliverBuf()
if (highestSeqNr < toOffset)
_toOffset = highestSeqNr
if (buf.isEmpty && (currOffset > toOffset || currOffset == fromOffset))
if (buf.isEmpty && (currOffset >= toOffset || currOffset == fromOffset))
onCompleteThenStop()
else
self ! Continue // more to fetch

View file

@ -116,6 +116,10 @@ class LeveldbReadJournal(scaladslReadJournal: akka.persistence.query.journal.lev
* `persistenceId` that persisted the event. The `persistenceId` + `sequenceNr` is an unique
* identifier for the event.
*
* The `offset` is exclusive, i.e. the event with the exact same sequence number will not be included
* in the returned stream. This means that you can use the offset that is returned in `EventEnvelope`
* as the `offset` parameter in a subsequent query.
*
* The returned event stream is ordered by the offset (tag sequence number), which corresponds
* to the same order as the write journal stored the events. The same stream elements (in same order)
* are returned for multiple executions of the query. Deleted events are not deleted from the

View file

@ -10,7 +10,7 @@ import akka.actor.ExtendedActorSystem
import akka.event.Logging
import akka.persistence.query.journal.leveldb.{ AllPersistenceIdsPublisher, EventsByPersistenceIdPublisher, EventsByTagPublisher }
import akka.persistence.query.scaladsl.{ ReadJournal, _ }
import akka.persistence.query.{ EventEnvelope, Offset, Sequence }
import akka.persistence.query.{ EventEnvelope, NoOffset, Offset, Sequence }
import akka.stream.scaladsl.Source
import akka.util.ByteString
import com.typesafe.config.Config
@ -128,12 +128,16 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
* To tag events you create an [[akka.persistence.journal.EventAdapter]] that wraps the events
* in a [[akka.persistence.journal.Tagged]] with the given `tags`.
*
* You can retrieve a subset of all events by specifying `offset`, or use `0L` to retrieve all
* events with a given tag. The `offset` corresponds to an ordered sequence number for
* You can use `NoOffset` to retrieve all events with a given tag or retrieve a subset of all
* events by specifying a `Sequence` `offset`. The `offset` corresponds to an ordered sequence number for
* the specific tag. Note that the corresponding offset of each event is provided in the
* [[akka.persistence.query.EventEnvelope]], which makes it possible to resume the
* stream at a later point from a given offset.
*
* The `offset` is exclusive, i.e. the event with the exact same sequence number will not be included
* in the returned stream. This means that you can use the offset that is returned in `EventEnvelope`
* as the `offset` parameter in a subsequent query.
*
* In addition to the `offset` the `EventEnvelope` also provides `persistenceId` and `sequenceNr`
* for each event. The `sequenceNr` is the sequence number for the persistent actor with the
* `persistenceId` that persisted the event. The `persistenceId` + `sequenceNr` is an unique
@ -163,7 +167,7 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
refreshInterval, maxBufSize, writeJournalPluginId))
.mapMaterializedValue(_ NotUsed)
.named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
case NoOffset eventsByTag(tag, Sequence(0L)) //recursive
case _
throw new IllegalArgumentException("LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
}
@ -179,7 +183,7 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
Source.actorPublisher[EventEnvelope](EventsByTagPublisher.props(tag, seq.value, Long.MaxValue,
None, maxBufSize, writeJournalPluginId)).mapMaterializedValue(_ NotUsed)
.named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
case NoOffset currentEventsByTag(tag, Sequence(0L)) //recursive
case _
throw new IllegalArgumentException("LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
}

View file

@ -13,6 +13,7 @@ import akka.stream.ActorMaterializer
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import akka.persistence.query.NoOffset
object EventsByTagSpec {
val config = """
@ -72,7 +73,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
b ! "a green leaf"
expectMsg(s"a green leaf-done")
val greenSrc = queries.currentEventsByTag(tag = "green", offset = Sequence(0L))
val greenSrc = queries.currentEventsByTag(tag = "green", offset = NoOffset)
greenSrc.runWith(TestSink.probe[Any])
.request(2)
.expectNext(EventEnvelope(Sequence(1L), "a", 2L, "a green apple"))
@ -109,11 +110,11 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
.expectComplete() // green cucumber not seen
}
"find events from offset" in {
"find events from offset (exclusive)" in {
val greenSrc = queries.currentEventsByTag(tag = "green", offset = Sequence(2L))
val probe = greenSrc.runWith(TestSink.probe[Any])
.request(10)
.expectNext(EventEnvelope(Sequence(2L), "a", 3L, "a green banana"))
// note that banana is not included, since exclusive offset
.expectNext(EventEnvelope(Sequence(3L), "b", 2L, "a green leaf"))
.expectNext(EventEnvelope(Sequence(4L), "c", 1L, "a green cucumber"))
.expectComplete()
@ -124,7 +125,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
"find new events" in {
val d = system.actorOf(TestActor.props("d"))
val blackSrc = queries.eventsByTag(tag = "black", offset = Sequence(0L))
val blackSrc = queries.eventsByTag(tag = "black", offset = NoOffset)
val probe = blackSrc.runWith(TestSink.probe[Any])
.request(2)
.expectNext(EventEnvelope(Sequence(1L), "b", 1L, "a black car"))
@ -142,11 +143,11 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
.expectNext(EventEnvelope(Sequence(3L), "d", 2L, "a black night"))
}
"find events from offset" in {
"find events from offset (exclusive)" in {
val greenSrc = queries.eventsByTag(tag = "green", offset = Sequence(2L))
val probe = greenSrc.runWith(TestSink.probe[Any])
.request(10)
.expectNext(EventEnvelope(Sequence(2L), "a", 3L, "a green banana"))
// note that banana is not included, since exclusive offset
.expectNext(EventEnvelope(Sequence(3L), "b", 2L, "a green leaf"))
.expectNext(EventEnvelope(Sequence(4L), "c", 1L, "a green cucumber"))
.expectNoMsg(100.millis)