+per #16541 initial version of the Persistence Query module
This commit is contained in:
parent
09aff42d40
commit
f849793f36
23 changed files with 1051 additions and 14 deletions
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.persistence.query
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
/**
|
||||
* A query hint that defines how to execute the query,
|
||||
* typically specific to the journal implementation.
|
||||
*
|
||||
* A plugin may optionally support a [[Hint]].
|
||||
*/
|
||||
trait Hint
|
||||
|
||||
/**
|
||||
* If the underlying datastore only supports queries that are completed when they reach the
|
||||
* end of the "result set", the journal has to submit new queries after a while in order
|
||||
* to support "infinite" event streams that include events stored after the initial query has completed.
|
||||
*
|
||||
* A plugin may optionally support this [[Hint]] for defining such a refresh interval.
|
||||
*/
|
||||
final case class RefreshInterval(interval: FiniteDuration) extends Hint
|
||||
|
||||
/**
|
||||
* Indicates that the event stream is supposed to be completed immediately when it
|
||||
* reaches the end of the "result set", as described in [[RefreshInterval]].
|
||||
*
|
||||
*/
|
||||
final case object NoRefresh extends NoRefresh {
|
||||
/** Java API */
|
||||
def getInstance: NoRefresh = this
|
||||
}
|
||||
sealed class NoRefresh extends Hint
|
||||
|
||||
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.persistence.query
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import akka.actor._
|
||||
import akka.event.Logging
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.Failure
|
||||
|
||||
/**
|
||||
* Persistence extension for queries.
|
||||
*/
|
||||
object PersistenceQuery extends ExtensionId[PersistenceQuery] with ExtensionIdProvider {
|
||||
/**
|
||||
* Java API.
|
||||
*/
|
||||
override def get(system: ActorSystem): PersistenceQuery = super.get(system)
|
||||
|
||||
def createExtension(system: ExtendedActorSystem): PersistenceQuery = new PersistenceQuery(system)
|
||||
|
||||
def lookup() = PersistenceQuery
|
||||
|
||||
/** INTERNAL API. */
|
||||
private[persistence] case class PluginHolder(plugin: scaladsl.ReadJournal) extends Extension
|
||||
|
||||
}
|
||||
|
||||
class PersistenceQuery(system: ExtendedActorSystem) extends Extension {
|
||||
import PersistenceQuery._
|
||||
|
||||
private val log = Logging(system, getClass)
|
||||
|
||||
/** Discovered query plugins. */
|
||||
private val readJournalPluginExtensionIds = new AtomicReference[Map[String, ExtensionId[PluginHolder]]](Map.empty)
|
||||
|
||||
/**
|
||||
* Returns the [[akka.persistence.query.scaladsl.ReadJournal]] specified by the given read journal configuration entry.
|
||||
*/
|
||||
@tailrec final def readJournalFor(readJournalPluginId: String): scaladsl.ReadJournal = {
|
||||
val configPath = readJournalPluginId
|
||||
val extensionIdMap = readJournalPluginExtensionIds.get
|
||||
extensionIdMap.get(configPath) match {
|
||||
case Some(extensionId) ⇒
|
||||
extensionId(system).plugin
|
||||
case None ⇒
|
||||
val extensionId = new ExtensionId[PluginHolder] {
|
||||
override def createExtension(system: ExtendedActorSystem): PluginHolder =
|
||||
PluginHolder(createPlugin(configPath))
|
||||
}
|
||||
readJournalPluginExtensionIds.compareAndSet(extensionIdMap, extensionIdMap.updated(configPath, extensionId))
|
||||
readJournalFor(readJournalPluginId) // Recursive invocation.
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*
|
||||
* Returns the [[akka.persistence.query.javadsl.ReadJournal]] specified by the given read journal configuration entry.
|
||||
*/
|
||||
final def getReadJournalFor(readJournalPluginId: String): javadsl.ReadJournal =
|
||||
new javadsl.ReadJournal(readJournalFor(readJournalPluginId))
|
||||
|
||||
private def createPlugin(configPath: String): scaladsl.ReadJournal = {
|
||||
require(!isEmpty(configPath) && system.settings.config.hasPath(configPath),
|
||||
s"'reference.conf' is missing persistence read journal plugin config path: '${configPath}'")
|
||||
val pluginActorName = configPath
|
||||
val pluginConfig = system.settings.config.getConfig(configPath)
|
||||
val pluginClassName = pluginConfig.getString("class")
|
||||
log.debug(s"Create plugin: ${pluginActorName} ${pluginClassName}")
|
||||
val pluginClass = system.dynamicAccess.getClassFor[AnyRef](pluginClassName).get
|
||||
|
||||
val plugin = system.dynamicAccess.createInstanceFor[scaladsl.ReadJournal](pluginClass, (classOf[ExtendedActorSystem], system) :: Nil)
|
||||
.orElse(system.dynamicAccess.createInstanceFor[scaladsl.ReadJournal](pluginClass, Nil))
|
||||
.recoverWith {
|
||||
case ex: Exception ⇒ Failure.apply(new IllegalArgumentException(s"Unable to create read journal plugin instance for path [$configPath], class [$pluginClassName]!", ex))
|
||||
}
|
||||
|
||||
// TODO possibly apply event adapters here
|
||||
plugin.get
|
||||
}
|
||||
|
||||
/** Check for default or missing identity. */
|
||||
private def isEmpty(text: String) = text == null || text.length == 0
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.persistence.query
|
||||
|
||||
/**
|
||||
* General interface for all queries. There are a few pre-defined queries,
|
||||
* such as [[EventsByPersistenceId]], [[AllPersistenceIds]] and [[EventsByTag]]
|
||||
* but implementation of these queries are optional. Query (journal) plugins
|
||||
* may define their own specialized queries.
|
||||
*
|
||||
* If a query plugin does not support a query it will return a stream that
|
||||
* will be completed with a failure of [[UnsupportedOperationException]].
|
||||
*/
|
||||
trait Query[T, M]
|
||||
|
||||
/**
|
||||
* Query all `PersistentActor` identifiers, i.e. as defined by the
|
||||
* `persistenceId` of the `PersistentActor`.
|
||||
*
|
||||
* A plugin may optionally support this [[Query]].
|
||||
*/
|
||||
final case object AllPersistenceIds extends Query[String, Unit]
|
||||
|
||||
/**
|
||||
* Query events for a specific `PersistentActor` identified by `persistenceId`.
|
||||
*
|
||||
* You can retrieve a subset of all events by specifying `fromSequenceNr` and `toSequenceNr`
|
||||
* or use `0L` and `Long.MaxValue` respectively to retrieve all events.
|
||||
*
|
||||
* The returned event stream should be ordered by sequence number.
|
||||
*
|
||||
* A plugin may optionally support this [[Query]].
|
||||
*/
|
||||
final case class EventsByPersistenceId(persistenceId: String, fromSequenceNr: Long = 0L, toSequenceNr: Long = Long.MaxValue)
|
||||
extends Query[Any, Unit]
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* A plugin may optionally support this [[Query]].
|
||||
*/
|
||||
final case class EventsByTag(tag: String, offset: Long = 0L) extends Query[EventEnvelope, Unit]
|
||||
|
||||
/**
|
||||
* Event wrapper adding meta data for the events in the result stream of
|
||||
* [[EventsByTag]] query, or similar queries.
|
||||
*/
|
||||
//#event-envelope
|
||||
final case class EventEnvelope(
|
||||
offset: Long,
|
||||
persistenceId: String,
|
||||
sequenceNr: Long,
|
||||
event: Any)
|
||||
//#event-envelope
|
||||
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query.javadsl
|
||||
|
||||
import akka.persistence.query.{ Query, Hint }
|
||||
import akka.stream.javadsl.Source
|
||||
|
||||
import scala.annotation.varargs
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*
|
||||
* API for reading persistent events and information derived
|
||||
* from stored persistent events.
|
||||
*
|
||||
* The purpose of the API is not to enforce compatibility between different
|
||||
* journal implementations, because the technical capabilities may be very different.
|
||||
* The interface is very open so that different journals may implement specific queries.
|
||||
*
|
||||
* Usage:
|
||||
* {{{
|
||||
* final ReadJournal journal =
|
||||
* PersistenceQuery.get(system).getReadJournalFor(queryPluginConfigPath);
|
||||
*
|
||||
* final Source<EventEnvelope, ?> events =
|
||||
* journal.query(new EventsByTag("mytag", 0L));
|
||||
* }}}
|
||||
*/
|
||||
|
||||
final class ReadJournal(backing: akka.persistence.query.scaladsl.ReadJournal) {
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*
|
||||
* A query that returns a `Source` with output type `T` and materialized value `M`.
|
||||
*
|
||||
* The `hints` are optional parameters that defines how to execute the
|
||||
* query, typically specific to the journal implementation.
|
||||
*
|
||||
*/
|
||||
@varargs def query[T, M](q: Query[T, M], hints: Hint*): Source[T, M] =
|
||||
backing.query(q, hints: _*).asJava
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query.scaladsl
|
||||
|
||||
import akka.persistence.query.{ Hint, Query }
|
||||
import akka.stream.scaladsl.Source
|
||||
|
||||
/**
|
||||
* API for reading persistent events and information derived
|
||||
* from stored persistent events.
|
||||
*
|
||||
* The purpose of the API is not to enforce compatibility between different
|
||||
* journal implementations, because the technical capabilities may be very different.
|
||||
* The interface is very open so that different journals may implement specific queries.
|
||||
*
|
||||
* Usage:
|
||||
* {{{
|
||||
* val journal = PersistenceQuery(system).readJournalFor(queryPluginConfigPath)
|
||||
* val events = journal.query(EventsByTag("mytag", 0L))
|
||||
* }}}
|
||||
*
|
||||
* For Java API see [[akka.persistence.query.javadsl.ReadJournal]].
|
||||
*/
|
||||
abstract class ReadJournal {
|
||||
|
||||
/**
|
||||
* A query that returns a `Source` with output type `T` and materialized
|
||||
* value `M`.
|
||||
*
|
||||
* The `hints` are optional parameters that defines how to execute the
|
||||
* query, typically specific to the journal implementation.
|
||||
*
|
||||
*/
|
||||
def query[T, M](q: Query[T, M], hints: Hint*): Source[T, M]
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query;
|
||||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.persistence.query.javadsl.ReadJournal;
|
||||
import akka.testkit.AkkaJUnitActorSystemResource;
|
||||
import org.junit.ClassRule;
|
||||
import scala.runtime.BoxedUnit;
|
||||
|
||||
public class PersistenceQueryTest {
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource =
|
||||
new AkkaJUnitActorSystemResource(PersistenceQueryTest.class.getName());
|
||||
|
||||
private final ActorSystem system = actorSystemResource.getSystem();
|
||||
|
||||
private final Hint hint = NoRefresh.getInstance();
|
||||
|
||||
// compile-only test
|
||||
@SuppressWarnings("unused")
|
||||
public void shouldExposeJavaDSLFriendlyQueryJournal() throws Exception {
|
||||
final ReadJournal readJournal = PersistenceQuery.get(system).getReadJournalFor("noop-journal");
|
||||
final akka.stream.javadsl.Source<EventEnvelope, BoxedUnit> tag = readJournal.query(new EventsByTag("tag", 0L), hint, hint); // java varargs
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query
|
||||
|
||||
import akka.stream.scaladsl.Source
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
/**
|
||||
* Use for tests only!
|
||||
* Emits infinite stream of strings (representing queried for events).
|
||||
*/
|
||||
class MockReadJournal extends scaladsl.ReadJournal {
|
||||
override def query[T, M](q: Query[T, M], hints: Hint*): Source[T, M] =
|
||||
Source(() ⇒ Iterator.from(0)).map(_.toString).asInstanceOf[Source[T, M]]
|
||||
}
|
||||
|
||||
object MockReadJournal {
|
||||
final val Identifier = "akka.persistence.query.journal.mock"
|
||||
|
||||
final val config: Config = ConfigFactory.parseString(
|
||||
s"""
|
||||
|$Identifier {
|
||||
| class = "${classOf[MockReadJournal].getCanonicalName}"
|
||||
|}
|
||||
""".stripMargin)
|
||||
}
|
||||
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.persistence.query
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.persistence.journal.{ EventAdapter, EventSeq }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpecLike }
|
||||
import org.scalautils.ConversionCheckedTripleEquals
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class PersistenceQuerySpec extends WordSpecLike with Matchers with BeforeAndAfterAll with ConversionCheckedTripleEquals {
|
||||
|
||||
val anything: Query[String, _] = null
|
||||
|
||||
val eventAdaptersConfig =
|
||||
s"""
|
||||
|akka.persistence.query.journal.mock {
|
||||
| event-adapters {
|
||||
| adapt = ${classOf[PrefixStringWithPAdapter].getCanonicalName}
|
||||
| }
|
||||
|}
|
||||
""".stripMargin
|
||||
|
||||
"ReadJournal" must {
|
||||
"be found by full config key" in {
|
||||
withActorSystem() { system ⇒
|
||||
PersistenceQuery.get(system).readJournalFor(MockReadJournal.Identifier)
|
||||
}
|
||||
}
|
||||
|
||||
"throw if unable to find query journal by config key" in {
|
||||
withActorSystem() { system ⇒
|
||||
intercept[IllegalArgumentException] {
|
||||
PersistenceQuery.get(system).readJournalFor(MockReadJournal.Identifier + "-unknown")
|
||||
}.getMessage should include("missing persistence read journal")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private val systemCounter = new AtomicInteger()
|
||||
private def withActorSystem(conf: String = "")(block: ActorSystem ⇒ Unit): Unit = {
|
||||
val config =
|
||||
MockReadJournal.config
|
||||
.withFallback(ConfigFactory.parseString(conf))
|
||||
.withFallback(ConfigFactory.parseString(eventAdaptersConfig))
|
||||
.withFallback(ConfigFactory.load())
|
||||
|
||||
val sys = ActorSystem(s"sys-${systemCounter.incrementAndGet()}", config)
|
||||
try block(sys) finally Await.ready(sys.terminate(), 10.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
object ExampleQueryModels {
|
||||
case class OldModel(value: String) { def promote = NewModel(value) }
|
||||
case class NewModel(value: String)
|
||||
}
|
||||
|
||||
class PrefixStringWithPAdapter extends EventAdapter {
|
||||
override def fromJournal(event: Any, manifest: String) = EventSeq.single("p-" + event)
|
||||
|
||||
override def manifest(event: Any) = ""
|
||||
override def toJournal(event: Any) = throw new Exception("toJournal should not be called by query side")
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue