diff --git a/.java-version b/.java-version index 810ee4e91e..6259340971 100644 --- a/.java-version +++ b/.java-version @@ -1 +1 @@ -1.6 +1.8 diff --git a/akka-docs/rst/java/code/docs/persistence/PersistencePluginDocTest.java b/akka-docs/rst/java/code/docs/persistence/PersistencePluginDocTest.java index f739aecfdb..a173191425 100644 --- a/akka-docs/rst/java/code/docs/persistence/PersistencePluginDocTest.java +++ b/akka-docs/rst/java/code/docs/persistence/PersistencePluginDocTest.java @@ -4,179 +4,191 @@ package docs.persistence; +import akka.actor.*; +import akka.dispatch.Futures; +import akka.persistence.*; +import akka.persistence.japi.journal.JavaJournalSpec; +import akka.persistence.japi.snapshot.JavaSnapshotStoreSpec; +import akka.persistence.journal.japi.AsyncWriteJournal; +import akka.persistence.journal.leveldb.SharedLeveldbJournal; +import akka.persistence.journal.leveldb.SharedLeveldbStore; +import akka.persistence.snapshot.japi.SnapshotStore; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import org.iq80.leveldb.util.FileUtils; +import scala.concurrent.Future; + import java.io.File; import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.function.Consumer; -import akka.actor.*; -import akka.dispatch.Futures; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import org.iq80.leveldb.util.FileUtils; -import akka.persistence.japi.journal.JavaJournalSpec; -import akka.persistence.japi.snapshot.JavaSnapshotStoreSpec; -import akka.persistence.journal.leveldb.SharedLeveldbJournal; -import akka.persistence.journal.leveldb.SharedLeveldbStore; -import scala.concurrent.Future; //#plugin-imports -import akka.persistence.*; -import akka.persistence.journal.japi.AsyncWriteJournal; -import akka.persistence.snapshot.japi.SnapshotStore; //#plugin-imports public class PersistencePluginDocTest { - static Object o1 = new Object() { - final ActorSystem system = null; - //#shared-store-creation - final ActorRef store = system.actorOf(Props.create(SharedLeveldbStore.class), "store"); - //#shared-store-creation + static Object o1 = new Object() { + final ActorSystem system = null; + //#shared-store-creation + final ActorRef store = system.actorOf(Props.create(SharedLeveldbStore.class), "store"); + //#shared-store-creation - //#shared-store-usage - class SharedStorageUsage extends UntypedActor { - @Override - public void preStart() throws Exception { - String path = "akka.tcp://example@127.0.0.1:2552/user/store"; - ActorSelection selection = getContext().actorSelection(path); - selection.tell(new Identify(1), getSelf()); + //#shared-store-usage + class SharedStorageUsage extends UntypedActor { + @Override + public void preStart() throws Exception { + String path = "akka.tcp://example@127.0.0.1:2552/user/store"; + ActorSelection selection = getContext().actorSelection(path); + selection.tell(new Identify(1), getSelf()); + } + + @Override + public void onReceive(Object message) throws Exception { + if (message instanceof ActorIdentity) { + ActorIdentity identity = (ActorIdentity) message; + if (identity.correlationId().equals(1)) { + ActorRef store = identity.getRef(); + if (store != null) { + SharedLeveldbJournal.setStore(store, getContext().system()); } - - @Override - public void onReceive(Object message) throws Exception { - if (message instanceof ActorIdentity) { - ActorIdentity identity = (ActorIdentity) message; - if (identity.correlationId().equals(1)) { - ActorRef store = identity.getRef(); - if (store != null) { - SharedLeveldbJournal.setStore(store, getContext().system()); - } - } - } - } - } - //#shared-store-usage - }; - - class MySnapshotStore extends SnapshotStore { - @Override - public Future> doLoadAsync(String persistenceId, SnapshotSelectionCriteria criteria) { - return null; - } - - @Override - public Future doSaveAsync(SnapshotMetadata metadata, Object snapshot) { - return null; - } - - @Override - public Future doDeleteAsync(SnapshotMetadata metadata) { - return Futures.successful(null); - } - - @Override - public Future doDeleteAsync(String persistenceId, SnapshotSelectionCriteria criteria) { - return Futures.successful(null); - } - } - - class MyAsyncJournal extends AsyncWriteJournal { - //#sync-journal-plugin-api - @Override - public Future>> doAsyncWriteMessages( - Iterable messages) { - try { - Iterable> result = new ArrayList>(); - // blocking call here... - // result.add(..) - return Futures.successful(result); - } catch (Exception e) { - return Futures.failed(e); } } - //#sync-journal-plugin-api + } + } + //#shared-store-usage + }; - @Override - public Future doAsyncDeleteMessagesTo(String persistenceId, long toSequenceNr) { - return null; - } - - @Override - public Future doAsyncReplayMessages(String persistenceId, long fromSequenceNr, - long toSequenceNr, long max, Consumer replayCallback) { - return null; - } - - @Override - public Future doAsyncReadHighestSequenceNr(String persistenceId, long fromSequenceNr) { - return null; - } + class MySnapshotStore extends SnapshotStore { + @Override + public Future> doLoadAsync(String persistenceId, SnapshotSelectionCriteria criteria) { + return null; } + @Override + public Future doSaveAsync(SnapshotMetadata metadata, Object snapshot) { + return null; + } - static Object o2 = new Object() { - //#journal-tck-java - class MyJournalSpecTest extends JavaJournalSpec { + @Override + public Future doDeleteAsync(SnapshotMetadata metadata) { + return Futures.successful(null); + } - public MyJournalSpecTest() { - super(ConfigFactory.parseString( - "persistence.journal.plugin = " + - "\"akka.persistence.journal.leveldb-shared\"")); - } + @Override + public Future doDeleteAsync(String persistenceId, SnapshotSelectionCriteria criteria) { + return Futures.successful(null); + } + } + + class MyAsyncJournal extends AsyncWriteJournal { + //#sync-journal-plugin-api + @Override + public Future>> doAsyncWriteMessages( + Iterable messages) { + try { + Iterable> result = new ArrayList>(); + // blocking call here... + // result.add(..) + return Futures.successful(result); + } catch (Exception e) { + return Futures.failed(e); + } + } + //#sync-journal-plugin-api + + @Override + public Future doAsyncDeleteMessagesTo(String persistenceId, long toSequenceNr) { + return null; + } + + @Override + public Future doAsyncReplayMessages(String persistenceId, long fromSequenceNr, + long toSequenceNr, long max, Consumer replayCallback) { + return null; + } + + @Override + public Future doAsyncReadHighestSequenceNr(String persistenceId, long fromSequenceNr) { + return null; + } + } + + + static Object o2 = new Object() { + //#journal-tck-java + class MyJournalSpecTest extends JavaJournalSpec { + + public MyJournalSpecTest() { + super(ConfigFactory.parseString( + "persistence.journal.plugin = " + + "\"akka.persistence.journal.leveldb-shared\"")); + } + + @Override + public CapabilityFlag supportsRejectingNonSerializableObjects() { + return CapabilityFlag.off(); + } + } + //#journal-tck-java + }; + + static Object o3 = new Object() { + //#snapshot-store-tck-java + class MySnapshotStoreTest extends JavaSnapshotStoreSpec { + + public MySnapshotStoreTest() { + super(ConfigFactory.parseString( + "akka.persistence.snapshot-store.plugin = " + + "\"akka.persistence.snapshot-store.local\"")); + } + } + //#snapshot-store-tck-java + }; + + static Object o4 = new Object() { + //#journal-tck-before-after-java + class MyJournalSpecTest extends JavaJournalSpec { + + List storageLocations = new ArrayList(); + + public MyJournalSpecTest() { + super(ConfigFactory.parseString( + "persistence.journal.plugin = " + + "\"akka.persistence.journal.leveldb-shared\"")); + + Config config = system().settings().config(); + storageLocations.add(new File( + config.getString("akka.persistence.journal.leveldb.dir"))); + storageLocations.add(new File( + config.getString("akka.persistence.snapshot-store.local.dir"))); + } + + + @Override + public CapabilityFlag supportsRejectingNonSerializableObjects() { + return CapabilityFlag.on(); + } + + @Override + public void beforeAll() { + for (File storageLocation : storageLocations) { + FileUtils.deleteRecursively(storageLocation); } - //#journal-tck-java - }; + super.beforeAll(); + } - static Object o3 = new Object() { - //#snapshot-store-tck-java - class MySnapshotStoreTest extends JavaSnapshotStoreSpec { - - public MySnapshotStoreTest() { - super(ConfigFactory.parseString( - "akka.persistence.snapshot-store.plugin = " + - "\"akka.persistence.snapshot-store.local\"")); - } + @Override + public void afterAll() { + super.afterAll(); + for (File storageLocation : storageLocations) { + FileUtils.deleteRecursively(storageLocation); } - //#snapshot-store-tck-java - }; - - static Object o4 = new Object() { - //#journal-tck-before-after-java - class MyJournalSpecTest extends JavaJournalSpec { - - List storageLocations = new ArrayList(); - - public MyJournalSpecTest() { - super(ConfigFactory.parseString( - "persistence.journal.plugin = " + - "\"akka.persistence.journal.leveldb-shared\"")); - - Config config = system().settings().config(); - storageLocations.add(new File( - config.getString("akka.persistence.journal.leveldb.dir"))); - storageLocations.add(new File( - config.getString("akka.persistence.snapshot-store.local.dir"))); - } - - @Override - public void beforeAll() { - for (File storageLocation : storageLocations) { - FileUtils.deleteRecursively(storageLocation); - } - super.beforeAll(); - } - - @Override - public void afterAll() { - super.afterAll(); - for (File storageLocation : storageLocations) { - FileUtils.deleteRecursively(storageLocation); - } - } - } - //#journal-tck-before-after-java - }; + } + } + //#journal-tck-before-after-java + }; } diff --git a/akka-docs/rst/java/persistence.rst b/akka-docs/rst/java/persistence.rst index d014d741bc..8491c8298b 100644 --- a/akka-docs/rst/java/persistence.rst +++ b/akka-docs/rst/java/persistence.rst @@ -814,6 +814,10 @@ To include the Journal TCK tests in your test suite simply extend the provided ` .. includecode:: ./code/docs/persistence/PersistencePluginDocTest.java#journal-tck-java +Please note that some of the tests are optional, and by overriding the ``supports...`` methods you give the +TCK the needed information about which tests to run. You can implement these methods using the provided +``CapabilityFlag.on`` / ``CapabilityFlag.off`` values. + We also provide a simple benchmarking class ``JavaJournalPerfSpec`` which includes all the tests that ``JavaJournalSpec`` has, and also performs some longer operations on the Journal while printing it's performance stats. While it is NOT aimed to provide a proper benchmarking environment it can be used to get a rough feel about your journals performance in the most diff --git a/akka-docs/rst/scala/code/docs/persistence/PersistencePluginDocSpec.scala b/akka-docs/rst/scala/code/docs/persistence/PersistencePluginDocSpec.scala index b0881583ff..2e9d85d212 100644 --- a/akka-docs/rst/scala/code/docs/persistence/PersistencePluginDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/persistence/PersistencePluginDocSpec.scala @@ -176,9 +176,11 @@ object PersistenceTCKDoc { //#journal-tck-scala class MyJournalSpec extends JournalSpec( config = ConfigFactory.parseString( - """ - akka.persistence.journal.plugin = "my.journal.plugin" - """)) + """akka.persistence.journal.plugin = "my.journal.plugin"""")) { + + override def supportsRejectingNonSerializableObjects: CapabilityFlag = + false // or CapabilityFlag.off + } //#journal-tck-scala } new AnyRef { @@ -205,6 +207,9 @@ object PersistenceTCKDoc { akka.persistence.journal.plugin = "my.journal.plugin" """)) { + override def supportsRejectingNonSerializableObjects: CapabilityFlag = + true // or CapabilityFlag.on + val storageLocations = List( new File(system.settings.config.getString("akka.persistence.journal.leveldb.dir")), new File(config.getString("akka.persistence.snapshot-store.local.dir"))) diff --git a/akka-docs/rst/scala/persistence.rst b/akka-docs/rst/scala/persistence.rst index f8fb7700be..1c86f7dd44 100644 --- a/akka-docs/rst/scala/persistence.rst +++ b/akka-docs/rst/scala/persistence.rst @@ -868,6 +868,10 @@ To include the Journal TCK tests in your test suite simply extend the provided ` .. includecode:: ./code/docs/persistence/PersistencePluginDocSpec.scala#journal-tck-scala +Please note that some of the tests are optional, and by overriding the ``supports...`` methods you give the +TCK the needed information about which tests to run. You can implement these methods using boolean falues or the +provided ``CapabilityFlag.on`` / ``CapabilityFlag.off`` values. + We also provide a simple benchmarking class ``JournalPerfSpec`` which includes all the tests that ``JournalSpec`` has, and also performs some longer operations on the Journal while printing it's performance stats. While it is NOT aimed to provide a proper benchmarking environment it can be used to get a rough feel about your journals performance in the most diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala new file mode 100644 index 0000000000..44bfae0209 --- /dev/null +++ b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2009-2015 Typesafe Inc. + */ + +package akka.persistence + +import scala.language.implicitConversions + +sealed abstract class CapabilityFlag { + private val capturedStack = (new Throwable().getStackTrace) + .filter(_.getMethodName.startsWith("supports")) + .find { el ⇒ + val clazz = Class.forName(el.getClassName) + clazz.getDeclaredMethod(el.getMethodName).getReturnType == classOf[CapabilityFlag] + } map { _.getMethodName } getOrElse "[unknown]" + + def name: String = capturedStack + def value: Boolean +} +object CapabilityFlag { + def on(): CapabilityFlag = + new CapabilityFlag { override def value = true } + def off(): CapabilityFlag = + new CapabilityFlag { override def value = true } + + /** Java DSL */ + def create(`val`: Boolean): CapabilityFlag = + new CapabilityFlag { override def value = `val` } + + // conversions + + implicit def mkFlag(v: Boolean): CapabilityFlag = + new CapabilityFlag { override def value = v } +} + +sealed trait CapabilityFlags + +//#journal-flags +trait JournalCapabilityFlags extends CapabilityFlags { + + /** + * When `true` enables tests which check if the Journal properly rejects + * writes of objects which are not `java.lang.Serializable`. + */ + protected def supportsRejectingNonSerializableObjects: CapabilityFlag + +} +//#journal-flags + +//#snapshot-store-flags +trait SnapshotStoreCapabilityFlags extends CapabilityFlags { + // no flags currently +} +//#snapshot-store-flags diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala index 9a13971417..cbca14fc65 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/PluginSpec.scala @@ -1,3 +1,6 @@ +/** + * Copyright (C) 2009-2015 Typesafe Inc. + */ package akka.persistence import java.util.concurrent.atomic.AtomicInteger diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala index 744bab944a..2536442df9 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalPerfSpec.scala @@ -3,7 +3,8 @@ */ package akka.persistence.japi.journal -import akka.persistence.journal.{ JournalPerfSpec, JournalSpec } +import akka.persistence.CapabilityFlag +import akka.persistence.journal.JournalPerfSpec import com.typesafe.config.Config import org.junit.runner.RunWith import org.scalatest.Informer @@ -47,4 +48,6 @@ class JavaJournalPerfSpec(config: Config) extends JournalPerfSpec(config) { override protected def info: Informer = new Informer { override def apply(message: String, payload: Option[Any]): Unit = System.out.println(message) } + + override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on } diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala index 9ff59fdca2..7f400109f4 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/journal/JavaJournalSpec.scala @@ -3,6 +3,7 @@ */ package akka.persistence.japi.journal +import akka.persistence.CapabilityFlag import akka.persistence.journal.JournalSpec import com.typesafe.config.Config import org.junit.runner.RunWith @@ -21,4 +22,6 @@ import org.scalatest.junit.JUnitRunner * @param config configures the Journal plugin to be tested */ @RunWith(classOf[JUnitRunner]) -class JavaJournalSpec(config: Config) extends JournalSpec(config) +class JavaJournalSpec(config: Config) extends JournalSpec(config) { + override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = CapabilityFlag.on +} diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala index 360629a535..3ebe07be19 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/japi/snapshot/JavaSnapshotStoreSpec.scala @@ -3,6 +3,7 @@ */ package akka.persistence.japi.snapshot +import akka.persistence.CapabilityFlag import akka.persistence.snapshot.{ SnapshotStore, SnapshotStoreSpec } import com.typesafe.config.Config import org.junit.runner.RunWith @@ -20,4 +21,4 @@ import org.scalatest.junit.JUnitRunner * @see [[akka.persistence.snapshot.SnapshotStoreSpec]] */ @RunWith(classOf[JUnitRunner]) -class JavaSnapshotStoreSpec(config: Config) extends SnapshotStoreSpec(config) +class JavaSnapshotStoreSpec(config: Config) extends SnapshotStoreSpec(config) \ No newline at end of file diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala index 154955510f..68f1917b21 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala @@ -1,5 +1,10 @@ +/** + * Copyright (C) 2009-2015 Typesafe Inc. + */ package akka.persistence.journal +import akka.persistence.scalatest.{ MayVerb, OptionalTests } + import scala.concurrent.duration._ import scala.collection.immutable.Seq @@ -29,7 +34,8 @@ object JournalSpec { * @see [[akka.persistence.journal.JournalPerfSpec]] * @see [[akka.persistence.japi.journal.JavaJournalPerfSpec]] */ -abstract class JournalSpec(config: Config) extends PluginSpec(config) { +abstract class JournalSpec(config: Config) extends PluginSpec(config) with MayVerb + with OptionalTests with JournalCapabilityFlags { implicit lazy val system: ActorSystem = ActorSystem("JournalSpec", config.withFallback(JournalSpec.config)) @@ -181,33 +187,39 @@ abstract class JournalSpec(config: Config) extends PluginSpec(config) { journal ! ReplayMessages(0, Long.MaxValue, Long.MaxValue, pid, receiverProbe.ref) receiverProbe.expectMsg(RecoverySuccess(highestSequenceNr = 5L)) } + } - "reject non-serializable events" in { - // there is no chance that a journal could create a data representation for type of event - val notSerializableEvent = new Object { override def toString = "not serializable" } - val msgs = (6 to 8).map { i ⇒ - val event = if (i == 7) notSerializableEvent else s"b-$i" - AtomicWrite(PersistentRepr(payload = event, sequenceNr = i, persistenceId = pid, sender = Actor.noSender, - writerUuid = writerUuid)) - } + "A Journal optionally" may { - val probe = TestProbe() - journal ! WriteMessages(msgs, probe.ref, actorInstanceId) + optional(flag = supportsRejectingNonSerializableObjects) { + "reject non-serializable events" in { + // there is no chance that a journal could create a data representation for type of event + val notSerializableEvent = new Object { + override def toString = "not serializable" + } + val msgs = (6 to 8).map { i ⇒ + val event = if (i == 7) notSerializableEvent else s"b-$i" + AtomicWrite(PersistentRepr(payload = event, sequenceNr = i, persistenceId = pid, sender = Actor.noSender, + writerUuid = writerUuid)) + } - probe.expectMsg(WriteMessagesSuccessful) - val Pid = pid - val WriterUuid = writerUuid - probe.expectMsgPF() { - case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid), _) ⇒ payload should be(s"b-6") - } - probe.expectMsgPF() { - case WriteMessageRejected(PersistentImpl(payload, 7L, Pid, _, _, Actor.noSender, WriterUuid), _, _) ⇒ - payload should be(notSerializableEvent) - } - probe.expectMsgPF() { - case WriteMessageSuccess(PersistentImpl(payload, 8L, Pid, _, _, Actor.noSender, WriterUuid), _) ⇒ payload should be(s"b-8") - } + val probe = TestProbe() + journal ! WriteMessages(msgs, probe.ref, actorInstanceId) + probe.expectMsg(WriteMessagesSuccessful) + val Pid = pid + val WriterUuid = writerUuid + probe.expectMsgPF() { + case WriteMessageSuccess(PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid), _) ⇒ payload should be(s"b-6") + } + probe.expectMsgPF() { + case WriteMessageRejected(PersistentImpl(payload, 7L, Pid, _, _, Actor.noSender, WriterUuid), _, _) ⇒ + payload should be(notSerializableEvent) + } + probe.expectMsgPF() { + case WriteMessageSuccess(PersistentImpl(payload, 8L, Pid, _, _, Actor.noSender, WriterUuid), _) ⇒ payload should be(s"b-8") + } + } } } } diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/MayVerb.scala b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/MayVerb.scala new file mode 100644 index 0000000000..ffceeff042 --- /dev/null +++ b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/MayVerb.scala @@ -0,0 +1,62 @@ +/** + * Copyright (C) 2014-2015 Typesafe Inc. + */ +package akka.persistence.scalatest + +import org.scalatest.exceptions.TestCanceledException +import org.scalatest.words.StringVerbBlockRegistration + +trait MayVerb { + import MayVerb._ + + /** + * Configurable number of frames to be shown when a MAY test fails (is canceled). + * + * Defaults to `3`. + * Must be geater than `0`. + */ + def mayVerbStacktraceContextFrames = 3 + + def optional(whenSkippedMessage: String)(body: ⇒ Unit): Unit = + try body catch { + case cause: Throwable ⇒ + val shortTrace = cause.getStackTrace.take(mayVerbStacktraceContextFrames) + throw new TestCanceledByFailure(whenSkippedMessage, shortTrace) + } + + trait StringMayWrapperForVerb { + val leftSideString: String + + /** + * Block of tests which MAY pass, and if not should be ignored. + * Such as rules which may be optionally implemented by Journals. + * + * MUST be used in conjunction with [[optional]] to provide explanation as to why it may be ok to fail this spec. + * + * The word `MAY` is to be understood as defined in RFC 2119 + * + * @see RFC 2119 + */ + def may(right: ⇒ Unit)(implicit fun: StringVerbBlockRegistration) { + fun(leftSideString, "may", right _) + } + } + + import scala.language.implicitConversions + + /** + * Implicitly converts an object of type String to a StringMayWrapper, + * to enable may methods to be invokable on that object. + */ + implicit def convertToStringMayWrapper(o: String): StringMayWrapperForVerb = + new StringMayWrapperForVerb { + val leftSideString = o.trim + } + +} + +object MayVerb { + case class TestCanceledByFailure(msg: String, specialStackTrace: Array[StackTraceElement]) extends TestCanceledException(Some(msg), None, 2) { + override def getStackTrace = specialStackTrace + } +} \ No newline at end of file diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala new file mode 100644 index 0000000000..a1d7ab4f3d --- /dev/null +++ b/akka-persistence-tck/src/main/scala/akka/persistence/scalatest/OptionalTests.scala @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2009-2015 Typesafe Inc. + */ + +package akka.persistence.scalatest + +import akka.persistence.CapabilityFlag +import org.scalatest.Informing + +trait OptionalTests { + this: Informing ⇒ + + def optional(flag: CapabilityFlag)(test: ⇒ Unit) = { + val msg = s"CapabilityFlag `${flag.name}` was turned `" + + (if (flag.value) "on" else "off") + + "`. " + (if (!flag.value) "To enable the related tests override it with `CapabilityFlag.on` (or `true` in Scala)." else "") + info(msg) + if (flag.value) + try test catch { + case ex: Exception ⇒ + throw new AssertionError("Imlpementation did not pass this spec. " + + "If your journal will be (by definition) unable to abide the here tested rule, you can disable this test," + + s"by overriding [${flag.name}] with CapabilityFlag.off in your test class.") + } + } + +} diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala index c6042e11df..4bd7d3674d 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala @@ -1,5 +1,10 @@ +/** + * Copyright (C) 2009-2015 Typesafe Inc. + */ package akka.persistence.snapshot +import akka.persistence.scalatest.OptionalTests + import scala.collection.immutable.Seq import akka.actor._ import akka.persistence._ @@ -23,7 +28,8 @@ object SnapshotStoreSpec { * * @see [[akka.persistence.japi.snapshot.JavaSnapshotStoreSpec]] */ -abstract class SnapshotStoreSpec(config: Config) extends PluginSpec(config) { +abstract class SnapshotStoreSpec(config: Config) extends PluginSpec(config) + with OptionalTests with SnapshotStoreCapabilityFlags { implicit lazy val system = ActorSystem("SnapshotStoreSpec", config.withFallback(SnapshotStoreSpec.config)) private var senderProbe: TestProbe = _ diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala index bc2a2ac615..f647080806 100644 --- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala +++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalJavaSpec.scala @@ -1,11 +1,14 @@ package akka.persistence.journal.leveldb import akka.persistence.journal.JournalSpec -import akka.persistence.{ PersistenceSpec, PluginCleanup } +import akka.persistence.{ CapabilityFlag, PersistenceSpec, PluginCleanup } class LeveldbJournalJavaSpec extends JournalSpec( config = PersistenceSpec.config( "leveldb", "LeveldbJournalJavaSpec", extraConfig = Some("akka.persistence.journal.leveldb.native = off"))) - with PluginCleanup + with PluginCleanup { + + override def supportsRejectingNonSerializableObjects = true +} diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala index 27455c1f53..db112f9e64 100644 --- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala +++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativePerfSpec.scala @@ -4,7 +4,7 @@ package akka.persistence.journal.leveldb import akka.persistence.journal.{ JournalPerfSpec, JournalSpec } -import akka.persistence.{ PersistenceSpec, PluginCleanup } +import akka.persistence.{ CapabilityFlag, PersistenceSpec, PluginCleanup } import org.scalatest.DoNotDiscover @DoNotDiscover // because only checking that compilation is OK with JournalPerfSpec @@ -13,4 +13,8 @@ class LeveldbJournalNativePerfSpec extends JournalPerfSpec( "leveldb", "LeveldbJournalNativePerfSpec", extraConfig = Some("akka.persistence.journal.leveldb.native = on"))) - with PluginCleanup + with PluginCleanup { + + override def supportsRejectingNonSerializableObjects = true + +} diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala index 3fc38a645b..b7d8ee641d 100644 --- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala +++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNativeSpec.scala @@ -8,4 +8,8 @@ class LeveldbJournalNativeSpec extends JournalSpec( "leveldb", "LeveldbJournalNativeSpec", extraConfig = Some("akka.persistence.journal.leveldb.native = on"))) - with PluginCleanup + with PluginCleanup { + + override def supportsRejectingNonSerializableObjects = true + +} diff --git a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala index 3e04471ce5..2a570d2516 100644 --- a/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala +++ b/akka-persistence-tck/src/test/scala/akka/persistence/journal/leveldb/LeveldbJournalNoAtomicPersistMultipleEventsSpec.scala @@ -14,5 +14,8 @@ class LeveldbJournalNoAtomicPersistMultipleEventsSpec extends JournalSpec( * Setting to false to test the single message atomic write behaviour of JournalSpec */ override def supportsAtomicPersistAllOfSeveralEvents = false + + override def supportsRejectingNonSerializableObjects = true + }