eventsByTag and currentEventsByTag to use Offset (#21615)
* EventsByTagQuery2 and CurrentEventsByTagQuery2 to keep binary compatibility
This commit is contained in:
parent
c3abde60d5
commit
375c032604
17 changed files with 247 additions and 64 deletions
|
|
@ -0,0 +1,23 @@
|
|||
/**
|
||||
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
trait Offset
|
||||
|
||||
final case class Sequence(val value: Long) extends Offset with Ordered[Sequence] {
|
||||
override def compare(that: Sequence): Int = value.compare(that.value)
|
||||
}
|
||||
|
||||
final case class TimeBasedUUID(val value: UUID) extends Offset with Ordered[TimeBasedUUID] {
|
||||
if (value == null || value.version != 1) {
|
||||
throw new IllegalArgumentException("UUID " + value + " is not a time-based UUID")
|
||||
}
|
||||
|
||||
override def compare(other: TimeBasedUUID): Int = value.compareTo(other.value)
|
||||
}
|
||||
|
||||
final case object NoOffset extends Offset
|
||||
|
|
@ -10,6 +10,7 @@ import akka.persistence.query.EventEnvelope
|
|||
/**
|
||||
* A plugin may optionally support this query by implementing this interface.
|
||||
*/
|
||||
@deprecated("To be replaced by CurrentEventsByTagQuery2 from Akka 2.5", "2.4.11")
|
||||
trait CurrentEventsByTagQuery extends ReadJournal {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,24 @@
|
|||
/**
|
||||
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.persistence.query.javadsl
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.stream.javadsl.Source
|
||||
|
||||
/**
|
||||
* A plugin may optionally support this query by implementing this interface.
|
||||
*/
|
||||
// TODO: Rename it to CurrentEventsByTagQuery in Akka 2.5
|
||||
trait CurrentEventsByTagQuery2 extends ReadJournal {
|
||||
|
||||
/**
|
||||
* Same type of query as [[EventsByTagQuery#eventsByTag]] but the event stream
|
||||
* is completed immediately when it reaches the end of the "result set". Events that are
|
||||
* stored after the query is completed are not included in the event stream.
|
||||
*/
|
||||
def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed]
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -10,6 +10,7 @@ import akka.persistence.query.EventEnvelope
|
|||
/**
|
||||
* A plugin may optionally support this query by implementing this interface.
|
||||
*/
|
||||
@deprecated("To be replaced by EventsByTagQuery2 from Akka 2.5", "2.4.11")
|
||||
trait EventsByTagQuery extends ReadJournal {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,41 @@
|
|||
/**
|
||||
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.persistence.query.javadsl
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.stream.javadsl.Source
|
||||
|
||||
/**
|
||||
* A plugin may optionally support this query by implementing this interface.
|
||||
*/
|
||||
// TODO: Rename it to EventsByTagQuery in Akka 2.5
|
||||
trait EventsByTagQuery2 extends ReadJournal {
|
||||
|
||||
/**
|
||||
* Query events that have a specific tag. A tag can for example correspond to an
|
||||
* aggregate root type (in DDD terminology).
|
||||
*
|
||||
* The consumer can keep track of its current position in the event stream by storing the
|
||||
* `offset` and restart the query from a given `offset` after a crash/restart.
|
||||
*
|
||||
* The exact meaning of the `offset` depends on the journal and must be documented by the
|
||||
* read journal plugin. It may be a sequential id number that uniquely identifies the
|
||||
* position of each event within the event stream. Distributed data stores cannot easily
|
||||
* support those semantics and they may use a weaker meaning. For example it may be a
|
||||
* timestamp (taken when the event was created or stored). Timestamps are not unique and
|
||||
* not strictly ordered, since clocks on different machines may not be synchronized.
|
||||
*
|
||||
* The returned event stream should be ordered by `offset` if possible, but this can also be
|
||||
* difficult to fulfill for a distributed data store. The order must be documented by the
|
||||
* read journal plugin.
|
||||
*
|
||||
* The stream is not completed when it reaches the end of the currently stored events,
|
||||
* but it continues to push new events when new events are persisted.
|
||||
* Corresponding query that is completed when it reaches the end of the currently
|
||||
* stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]].
|
||||
*/
|
||||
def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed]
|
||||
|
||||
}
|
||||
|
|
@ -5,7 +5,7 @@ package akka.persistence.query.journal.leveldb.javadsl
|
|||
|
||||
import akka.NotUsed
|
||||
|
||||
import akka.persistence.query.EventEnvelope
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.persistence.query.javadsl._
|
||||
import akka.stream.javadsl.Source
|
||||
|
||||
|
|
@ -32,7 +32,9 @@ class LeveldbReadJournal(scaladslReadJournal: akka.persistence.query.journal.lev
|
|||
with EventsByPersistenceIdQuery
|
||||
with CurrentEventsByPersistenceIdQuery
|
||||
with EventsByTagQuery
|
||||
with CurrentEventsByTagQuery {
|
||||
with EventsByTagQuery2
|
||||
with CurrentEventsByTagQuery
|
||||
with CurrentEventsByTagQuery2 {
|
||||
|
||||
/**
|
||||
* `allPersistenceIds` is used for retrieving all `persistenceIds` of all
|
||||
|
|
@ -137,6 +139,9 @@ class LeveldbReadJournal(scaladslReadJournal: akka.persistence.query.journal.lev
|
|||
* The stream is completed with failure if there is a failure in executing the query in the
|
||||
* backend journal.
|
||||
*/
|
||||
override def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] =
|
||||
scaladslReadJournal.eventsByTag(tag, offset).asJava
|
||||
|
||||
override def eventsByTag(tag: String, offset: Long): Source[EventEnvelope, NotUsed] =
|
||||
scaladslReadJournal.eventsByTag(tag, offset).asJava
|
||||
|
||||
|
|
@ -145,6 +150,9 @@ class LeveldbReadJournal(scaladslReadJournal: akka.persistence.query.journal.lev
|
|||
* is completed immediately when it reaches the end of the "result set". Events that are
|
||||
* stored after the query is completed are not included in the event stream.
|
||||
*/
|
||||
override def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed] =
|
||||
scaladslReadJournal.currentEventsByTag(tag, offset).asJava
|
||||
|
||||
override def currentEventsByTag(tag: String, offset: Long): Source[EventEnvelope, NotUsed] =
|
||||
scaladslReadJournal.currentEventsByTag(tag, offset).asJava
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,9 +8,8 @@ import java.net.URLEncoder
|
|||
import akka.NotUsed
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.persistence.query.EventEnvelope
|
||||
import akka.persistence.query.{ EventEnvelope, Offset, Sequence }
|
||||
import akka.persistence.query.journal.leveldb.AllPersistenceIdsPublisher
|
||||
import akka.persistence.query.journal.leveldb.EventsByPersistenceIdPublisher
|
||||
import akka.persistence.query.journal.leveldb.EventsByTagPublisher
|
||||
|
|
@ -41,7 +40,9 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
|
|||
with EventsByPersistenceIdQuery
|
||||
with CurrentEventsByPersistenceIdQuery
|
||||
with EventsByTagQuery
|
||||
with CurrentEventsByTagQuery {
|
||||
with EventsByTagQuery2
|
||||
with CurrentEventsByTagQuery
|
||||
with CurrentEventsByTagQuery2 {
|
||||
|
||||
private val serialization = SerializationExtension(system)
|
||||
private val refreshInterval = Some(config.getDuration("refresh-interval", MILLISECONDS).millis)
|
||||
|
|
@ -165,7 +166,17 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
|
|||
* The stream is completed with failure if there is a failure in executing the query in the
|
||||
* backend journal.
|
||||
*/
|
||||
override def eventsByTag(tag: String, offset: Long = 0L): Source[EventEnvelope, NotUsed] = {
|
||||
override def eventsByTag(tag: String, offset: Offset = Sequence(0L)): Source[EventEnvelope, NotUsed] =
|
||||
offset match {
|
||||
case Sequence(offsetValue) ⇒
|
||||
Source.actorPublisher[EventEnvelope](EventsByTagPublisher.props(tag, offsetValue, Long.MaxValue,
|
||||
refreshInterval, maxBufSize, writeJournalPluginId)).mapMaterializedValue(_ ⇒ NotUsed)
|
||||
.named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("LevelDB does not support " + offset.getClass.getName + " offsets")
|
||||
}
|
||||
|
||||
override def eventsByTag(tag: String, offset: Long): Source[EventEnvelope, NotUsed] = {
|
||||
Source.actorPublisher[EventEnvelope](EventsByTagPublisher.props(tag, offset, Long.MaxValue,
|
||||
refreshInterval, maxBufSize, writeJournalPluginId)).mapMaterializedValue(_ ⇒ NotUsed)
|
||||
.named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
|
||||
|
|
@ -176,7 +187,17 @@ class LeveldbReadJournal(system: ExtendedActorSystem, config: Config) extends Re
|
|||
* is completed immediately when it reaches the end of the "result set". Events that are
|
||||
* stored after the query is completed are not included in the event stream.
|
||||
*/
|
||||
override def currentEventsByTag(tag: String, offset: Long = 0L): Source[EventEnvelope, NotUsed] = {
|
||||
override def currentEventsByTag(tag: String, offset: Offset = Sequence(0L)): Source[EventEnvelope, NotUsed] =
|
||||
offset match {
|
||||
case Sequence(offsetValue) ⇒
|
||||
Source.actorPublisher[EventEnvelope](EventsByTagPublisher.props(tag, offsetValue, Long.MaxValue,
|
||||
None, maxBufSize, writeJournalPluginId)).mapMaterializedValue(_ ⇒ NotUsed)
|
||||
.named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("LevelDB does not support " + offset.getClass.getName + " offsets")
|
||||
}
|
||||
|
||||
override def currentEventsByTag(tag: String, offset: Long): Source[EventEnvelope, NotUsed] = {
|
||||
Source.actorPublisher[EventEnvelope](EventsByTagPublisher.props(tag, offset, Long.MaxValue,
|
||||
None, maxBufSize, writeJournalPluginId)).mapMaterializedValue(_ ⇒ NotUsed)
|
||||
.named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.persistence.query.EventEnvelope
|
|||
/**
|
||||
* A plugin may optionally support this query by implementing this trait.
|
||||
*/
|
||||
@deprecated("To be replaced by CurrentEventsByTagQuery2 from Akka 2.5", "2.4.11")
|
||||
trait CurrentEventsByTagQuery extends ReadJournal {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,24 @@
|
|||
/**
|
||||
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.persistence.query.scaladsl
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.stream.scaladsl.Source
|
||||
|
||||
/**
|
||||
* A plugin may optionally support this query by implementing this trait.
|
||||
*/
|
||||
// TODO: Rename it to CurrentEventsByTagQuery in Akka 2.5
|
||||
trait CurrentEventsByTagQuery2 extends ReadJournal {
|
||||
|
||||
/**
|
||||
* Same type of query as [[EventsByTagQuery#eventsByTag]] but the event stream
|
||||
* is completed immediately when it reaches the end of the "result set". Events that are
|
||||
* stored after the query is completed are not included in the event stream.
|
||||
*/
|
||||
def currentEventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed]
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -10,6 +10,7 @@ import akka.persistence.query.EventEnvelope
|
|||
/**
|
||||
* A plugin may optionally support this query by implementing this trait.
|
||||
*/
|
||||
@deprecated("To be replaced by EventsByTagQuery2 from Akka 2.5", "2.4.11")
|
||||
trait EventsByTagQuery extends ReadJournal {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
package akka.persistence.query.scaladsl
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.persistence.query.{ EventEnvelope, Offset }
|
||||
import akka.stream.scaladsl.Source
|
||||
|
||||
/**
|
||||
* A plugin may optionally support this query by implementing this trait.
|
||||
*/
|
||||
// TODO: Rename it to EventsByTagQuery in Akka 2.5
|
||||
trait EventsByTagQuery2 extends ReadJournal {
|
||||
|
||||
/**
|
||||
* Query events that have a specific tag. A tag can for example correspond to an
|
||||
* aggregate root type (in DDD terminology).
|
||||
*
|
||||
* The consumer can keep track of its current position in the event stream by storing the
|
||||
* `offset` and restart the query from a given `offset` after a crash/restart.
|
||||
*
|
||||
* The exact meaning of the `offset` depends on the journal and must be documented by the
|
||||
* read journal plugin. It may be a sequential id number that uniquely identifies the
|
||||
* position of each event within the event stream. Distributed data stores cannot easily
|
||||
* support those semantics and they may use a weaker meaning. For example it may be a
|
||||
* timestamp (taken when the event was created or stored). Timestamps are not unique and
|
||||
* not strictly ordered, since clocks on different machines may not be synchronized.
|
||||
*
|
||||
* The returned event stream should be ordered by `offset` if possible, but this can also be
|
||||
* difficult to fulfill for a distributed data store. The order must be documented by the
|
||||
* read journal plugin.
|
||||
*
|
||||
* The stream is not completed when it reaches the end of the currently stored events,
|
||||
* but it continues to push new events when new events are persisted.
|
||||
* Corresponding query that is completed when it reaches the end of the currently
|
||||
* stored events is provided by [[CurrentEventsByTagQuery#currentEventsByTag]].
|
||||
*/
|
||||
def eventsByTag(tag: String, offset: Offset): Source[EventEnvelope, NotUsed]
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.duration._
|
|||
import akka.actor.ActorRef
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
|
||||
import akka.persistence.query.scaladsl.EventsByTagQuery
|
||||
import akka.persistence.query.scaladsl.EventsByTagQuery2
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.testkit.AkkaSpec
|
||||
|
|
@ -49,7 +49,7 @@ class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.confi
|
|||
"Leveldb query EventsByPersistenceId" must {
|
||||
|
||||
"implement standard EventsByTagQuery" in {
|
||||
queries.isInstanceOf[EventsByTagQuery] should ===(true)
|
||||
queries.isInstanceOf[EventsByTagQuery2] should ===(true)
|
||||
}
|
||||
|
||||
"find existing events" in {
|
||||
|
|
|
|||
|
|
@ -4,13 +4,11 @@
|
|||
package akka.persistence.query.journal.leveldb
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.persistence.journal.Tagged
|
||||
import akka.persistence.journal.WriteEventAdapter
|
||||
import akka.persistence.query.EventEnvelope
|
||||
import akka.persistence.query.PersistenceQuery
|
||||
import akka.persistence.query.{ EventEnvelope, PersistenceQuery, Sequence }
|
||||
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
|
||||
import akka.persistence.query.scaladsl.EventsByTagQuery
|
||||
import akka.persistence.query.scaladsl.EventsByTagQuery2
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.testkit.AkkaSpec
|
||||
|
|
@ -57,7 +55,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
|
||||
"Leveldb query EventsByTag" must {
|
||||
"implement standard EventsByTagQuery" in {
|
||||
queries.isInstanceOf[EventsByTagQuery] should ===(true)
|
||||
queries.isInstanceOf[EventsByTagQuery2] should ===(true)
|
||||
}
|
||||
|
||||
"find existing events" in {
|
||||
|
|
@ -74,7 +72,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
b ! "a green leaf"
|
||||
expectMsg(s"a green leaf-done")
|
||||
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = 0L)
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = Sequence(0L))
|
||||
greenSrc.runWith(TestSink.probe[Any])
|
||||
.request(2)
|
||||
.expectNext(EventEnvelope(1L, "a", 2L, "a green apple"))
|
||||
|
|
@ -84,7 +82,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
.expectNext(EventEnvelope(3L, "b", 2L, "a green leaf"))
|
||||
.expectComplete()
|
||||
|
||||
val blackSrc = queries.currentEventsByTag(tag = "black", offset = 0L)
|
||||
val blackSrc = queries.currentEventsByTag(tag = "black", offset = Sequence(0L))
|
||||
blackSrc.runWith(TestSink.probe[Any])
|
||||
.request(5)
|
||||
.expectNext(EventEnvelope(1L, "b", 1L, "a black car"))
|
||||
|
|
@ -94,7 +92,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
"not see new events after demand request" in {
|
||||
val c = system.actorOf(TestActor.props("c"))
|
||||
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = 0L)
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = Sequence(0L))
|
||||
val probe = greenSrc.runWith(TestSink.probe[Any])
|
||||
.request(2)
|
||||
.expectNext(EventEnvelope(1L, "a", 2L, "a green apple"))
|
||||
|
|
@ -112,7 +110,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
}
|
||||
|
||||
"find events from offset" in {
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = 2L)
|
||||
val greenSrc = queries.currentEventsByTag(tag = "green", offset = Sequence(2L))
|
||||
val probe = greenSrc.runWith(TestSink.probe[Any])
|
||||
.request(10)
|
||||
.expectNext(EventEnvelope(2L, "a", 3L, "a green banana"))
|
||||
|
|
@ -126,7 +124,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 = 0L)
|
||||
val blackSrc = queries.eventsByTag(tag = "black", offset = Sequence(0L))
|
||||
val probe = blackSrc.runWith(TestSink.probe[Any])
|
||||
.request(2)
|
||||
.expectNext(EventEnvelope(1L, "b", 1L, "a black car"))
|
||||
|
|
@ -145,7 +143,7 @@ class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config)
|
|||
}
|
||||
|
||||
"find events from offset" in {
|
||||
val greenSrc = queries.eventsByTag(tag = "green", offset = 2L)
|
||||
val greenSrc = queries.eventsByTag(tag = "green", offset = Sequence(2L))
|
||||
val probe = greenSrc.runWith(TestSink.probe[Any])
|
||||
.request(10)
|
||||
.expectNext(EventEnvelope(2L, "a", 3L, "a green banana"))
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue