+per #16541 initial version of the Persistence Query module

This commit is contained in:
Patrik Nordwall & Konrad Malawski 2015-06-08 12:26:19 +02:00 committed by Konrad Malawski
parent 09aff42d40
commit f849793f36
23 changed files with 1051 additions and 14 deletions

View file

@ -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

View file

@ -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
}

View file

@ -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

View file

@ -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&lt;EventEnvelope, ?&gt; 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
}

View file

@ -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]
}

View file

@ -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
}
}

View file

@ -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)
}

View file

@ -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")
}