diff --git a/project/Formatting.scala b/project/Formatting.scala index 2acea16b..890b144b 100644 --- a/project/Formatting.scala +++ b/project/Formatting.scala @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.rbmhtechnology.eventuate import sbt._ diff --git a/src/it/scala/com/rbmhtechnology/eventuate/EventsourcedProcessorIntegrationSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/EventsourcedProcessorIntegrationSpec.scala index c9ad385f..1b53e834 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/EventsourcedProcessorIntegrationSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/EventsourcedProcessorIntegrationSpec.scala @@ -24,7 +24,6 @@ import com.rbmhtechnology.eventuate.log.EventLogLifecycleLeveldb import org.scalatest._ -import scala.collection.immutable.Seq import scala.util._ object EventsourcedProcessorIntegrationSpec { diff --git a/src/it/scala/com/rbmhtechnology/eventuate/RecoverySpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/RecoverySpec.scala index aa3a2882..c1c9fa46 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/RecoverySpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/RecoverySpec.scala @@ -24,7 +24,7 @@ import akka.testkit.TestProbe import akka.util.Timeout import com.rbmhtechnology.eventuate.log._ -import com.rbmhtechnology.eventuate.log.leveldb.LeveldbSettings +import com.rbmhtechnology.eventuate.log.leveldb.LeveldbEventLogSettings import com.rbmhtechnology.eventuate.utilities._ import org.apache.commons.io.FileUtils @@ -57,7 +57,7 @@ object RecoverySpec { """.stripMargin def rootDirectory(target: ReplicationTarget): File = - new File(new LeveldbSettings(target.endpoint.system).rootDir) + new File(new LeveldbEventLogSettings(target.endpoint.system.settings.config).rootDir) def logDirectory(target: ReplicationTarget): File = { implicit val timeout = Timeout(3.seconds) diff --git a/src/it/scala/com/rbmhtechnology/eventuate/crdt/CRDTChaosSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/crdt/CRDTChaosSpec.scala index b298e57b..cdc83ed0 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/crdt/CRDTChaosSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/crdt/CRDTChaosSpec.scala @@ -8,12 +8,12 @@ import akka.testkit.TestProbe import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.crdt.CRDTService.ValueUpdated +import com.rbmhtechnology.eventuate.log.EventLog.Clock import com.rbmhtechnology.eventuate.log._ import com.rbmhtechnology.eventuate.log.cassandra._ import com.rbmhtechnology.eventuate.log.leveldb._ import org.cassandraunit.utils.EmbeddedCassandraServerHelper -import org.iq80.leveldb.WriteBatch import org.scalatest._ import scala.collection.immutable.Seq @@ -128,8 +128,8 @@ class CRDTChaosSpecLeveldb extends CRDTChaosSpec with EventLogCleanupLeveldb { import CRDTChaosSpec._ class TestEventLog(id: String) extends LeveldbEventLog(id, "log-test") { - private[eventuate] override def write(events: Seq[DurableEvent], tracker: TimeTracker, batch: WriteBatch): TimeTracker = - if (events.map(_.payload).contains(ValueUpdated(crdtId, AddOp(randomNr())))) throw boom else super.write(events, tracker, batch) + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = + if (events.map(_.payload).contains(ValueUpdated(crdtId, AddOp(randomNr())))) throw boom else super.write(events, partition, clock) } val logFactory: String => Props = @@ -137,7 +137,7 @@ class CRDTChaosSpecLeveldb extends CRDTChaosSpec with EventLogCleanupLeveldb { def logProps(logId: String, batching: Boolean = true): Props = { val logProps = Props(new TestEventLog(logId)).withDispatcher("eventuate.log.leveldb.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + if (batching) Props(new BatchingLayer(logProps)) else logProps } } @@ -145,8 +145,8 @@ class CRDTChaosSpecCassandra extends CRDTChaosSpec with EventLogCleanupCassandr import CRDTChaosSpec._ class TestEventLog(id: String) extends CassandraEventLog(id) { - private[eventuate] override def write(partition: Long, events: Seq[DurableEvent], tracker: TimeTracker): TimeTracker = - if (events.map(_.payload).contains(ValueUpdated(crdtId, AddOp(randomNr())))) throw boom else super.write(partition, events, tracker) + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = + if (events.map(_.payload).contains(ValueUpdated(crdtId, AddOp(randomNr())))) throw boom else super.write(events, partition, clock) } override val logFactory: String => Props = @@ -165,6 +165,6 @@ class CRDTChaosSpecCassandra extends CRDTChaosSpec with EventLogCleanupCassandr def logProps(logId: String, batching: Boolean = true): Props = { val logProps = Props(new TestEventLog(logId)).withDispatcher("eventuate.log.cassandra.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + if (batching) Props(new BatchingLayer(logProps)) else logProps } } diff --git a/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogLifecycle.scala b/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogLifecycle.scala index d57611b2..4c9484f9 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogLifecycle.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogLifecycle.scala @@ -19,27 +19,21 @@ package com.rbmhtechnology.eventuate.log import java.io.{Closeable, File} import akka.actor._ -import akka.pattern.ask import akka.testkit.{TestProbe, TestKit} -import akka.util.Timeout import com.rbmhtechnology.eventuate._ -import com.rbmhtechnology.eventuate.ReplicationProtocol._ +import com.rbmhtechnology.eventuate.log.EventLog._ import com.rbmhtechnology.eventuate.log.cassandra._ import com.rbmhtechnology.eventuate.log.cassandra.CassandraIndex._ -import com.rbmhtechnology.eventuate.log.leveldb.LeveldbEventLog -import com.rbmhtechnology.eventuate.log.leveldb.LeveldbEventLog.ReadResult +import com.rbmhtechnology.eventuate.log.leveldb._ import com.typesafe.config.Config import org.apache.commons.io.FileUtils import org.cassandraunit.utils.EmbeddedCassandraServerHelper -import org.iq80.leveldb.WriteBatch import org.scalatest._ import scala.collection.immutable.Seq import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration._ -import scala.util._ trait EventLogCleanupLeveldb extends Suite with BeforeAndAfterAll { def config: Config @@ -96,25 +90,22 @@ trait EventLogLifecycleLeveldb extends EventLogCleanupLeveldb with BeforeAndAfte object EventLogLifecycleLeveldb { object TestEventLog { def props(logId: String, batching: Boolean): Props = { - val logProps = Props(new EventLogLifecycleLeveldb.TestEventLog(logId)).withDispatcher("eventuate.log.leveldb.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + val logProps = Props(new TestEventLog(logId)).withDispatcher("eventuate.log.leveldb.write-dispatcher") + if (batching) Props(new BatchingLayer(logProps)) else logProps } } class TestEventLog(id: String) extends LeveldbEventLog(id, "log-test") { - override def currentSystemTime: Long = - 0L + override def currentSystemTime: Long = 0L - private[eventuate] override def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable, from: Long): ActorRef = - if (from == -1L) super.replayer(requestor, EventLogLifecycle.failingEventIterator, from) else super.replayer(requestor, iterator, from) + override def eventIterator(parameters: LeveldbEventIteratorParameters): Iterator[DurableEvent] with Closeable = + if (parameters.fromSequenceNr == -1L) EventLogLifecycle.failingEventIterator else super.eventIterator(parameters) - private[eventuate] override def read(from: Long, max: Int, filter: ReplicationFilter, lower: VectorTime): ReadResult = - if (from == -1L) throw boom else super.read(from, max, filter, lower) + override def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: (DurableEvent) => Boolean): Future[EventLog.ReadResult] = + if (fromSequenceNr == -1L) Future.failed(boom) else super.read(fromSequenceNr, toSequenceNr, max, filter) - private[eventuate] override def write(events: Seq[DurableEvent], tracker: TimeTracker, batch: WriteBatch): TimeTracker = events match { - case es if es.map(_.payload).contains("boom") => throw boom - case _ => super.write(events, tracker, batch) - } + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = + if (events.map(_.payload).contains("boom")) throw boom else super.write(events, partition, clock) override def unhandled(message: Any): Unit = message match { case "boom" => @@ -210,94 +201,64 @@ object EventLogLifecycleCassandra { def props(logId: String, failureSpec: TestFailureSpec, indexProbe: Option[ActorRef], batching: Boolean): Props = { val logProps = Props(new TestEventLog(logId, failureSpec, indexProbe)).withDispatcher("eventuate.log.cassandra.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + if (batching) Props(new BatchingLayer(logProps)) else logProps } } class TestEventLog(id: String, failureSpec: TestFailureSpec, indexProbe: Option[ActorRef]) extends CassandraEventLog(id) { - import context.dispatcher - private var index: ActorRef = _ - override def currentSystemTime: Long = - 0L + override def currentSystemTime: Long = 0L override def unhandled(message: Any): Unit = message match { - case GetReplicationProgresses => - val sdr = sender() - getReplicationProgressMap(List(EventLogSpec.remoteLogId, "x", "y")) onComplete { - case Success(r) => sdr ! GetReplicationProgressesSuccess(r) - case Failure(e) => sdr ! GetReplicationProgressesFailure(e) - } case "boom" => throw boom case _ => super.unhandled(message) } - private[eventuate] override def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable, from: Long): ActorRef = - if (from == -1L) super.replayer(requestor, EventLogLifecycle.failingEventIterator, from) else super.replayer(requestor, iterator, from) - - private[eventuate] override def write(partition: Long, events: Seq[DurableEvent], tracker: TimeTracker): TimeTracker = events match { - case es if es.map(_.payload).contains("boom") => throw boom - case _ => super.write(partition, events, tracker) - } - - private[eventuate] override def createEventReader(cassandra: Cassandra, logId: String) = - new TestEventReader(cassandra, logId) - - private[eventuate] override def createIndex(cassandra: Cassandra, eventReader: CassandraEventReader, logId: String): ActorRef = { - index = context.actorOf(Props(new TestIndex(cassandra, eventReader, logId, failureSpec, indexProbe))) - index - } - - private def getReplicationProgressMap(sourceLogIds: Seq[String]): Future[Map[String, Long]] = { - implicit val timeout = Timeout(10.seconds) + override def eventIterator(parameters: CassandraEventIteratorParameters): Iterator[DurableEvent] with Closeable = + if (parameters.fromSequenceNr == -1L) EventLogLifecycle.failingEventIterator else super.eventIterator(parameters) - Future.sequence(sourceLogIds.map(sourceLogId => index.ask(GetReplicationProgress(sourceLogId)).mapTo[GetReplicationProgressSuccess])).map { results => - results.foldLeft[Map[String, Long]](Map.empty) { - case (acc, GetReplicationProgressSuccess(logId, snr, _)) => if (snr == 0L) acc else acc + (logId -> snr) - } - } - } - } + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = + if (events.map(_.payload).contains("boom")) throw boom else super.write(events, partition, clock) - class TestEventReader(cassandra: Cassandra, logId: String) extends CassandraEventReader(cassandra, logId) { - override def read(from: Long, to: Long, max: Int, filter: ReplicationFilter, lower: VectorTime, targetLogId: String): CassandraEventReader.ReadResult = - if (from == -1L) throw boom else super.read(from, to: Long, max, filter, lower, targetLogId) - } - - class TestIndex(cassandra: Cassandra, eventReader: CassandraEventReader, logId: String, failureSpec: TestFailureSpec, indexProbe: Option[ActorRef]) extends CassandraIndex(cassandra, eventReader, logId) { - val stream = context.system.eventStream + private[eventuate] override def createEventLogStore(cassandra: Cassandra, logId: String) = + new TestEventLogStore(cassandra, logId) private[eventuate] override def createIndexStore(cassandra: Cassandra, logId: String) = new TestIndexStore(cassandra, logId, failureSpec) - override def onIndexEvent(event: Any): Unit = + private[eventuate] override def onIndexEvent(event: Any): Unit = indexProbe.foreach(_ ! event) } + class TestEventLogStore(cassandra: Cassandra, logId: String) extends CassandraEventLogStore(cassandra, logId) { + override def read(from: Long, to: Long, max: Int, filter: DurableEvent => Boolean): ReadResult = + if (from == -1L) throw boom else super.read(from, to: Long, max, filter) + } + class TestIndexStore(cassandra: Cassandra, logId: String, failureSpec: TestFailureSpec) extends CassandraIndexStore(cassandra, logId) { private var writeIndexIncrementFailed = false private var readSequenceNumberFailed = false - private[eventuate] override def writeAsync(aggregateEvents: AggregateEvents, timeTracker: TimeTracker)(implicit executor: ExecutionContext): Future[TimeTracker] = + private[eventuate] override def writeAsync(aggregateEvents: AggregateEvents, clock: Clock)(implicit executor: ExecutionContext): Future[Clock] = if (failureSpec.failBeforeIndexIncrementWrite && !writeIndexIncrementFailed) { writeIndexIncrementFailed = true Future.failed(boom) } else if (failureSpec.failAfterIndexIncrementWrite && !writeIndexIncrementFailed) { writeIndexIncrementFailed = true for { - _ <- super.writeAsync(aggregateEvents, timeTracker) + _ <- super.writeAsync(aggregateEvents, clock) r <- Future.failed(boom) } yield r - } else super.writeAsync(aggregateEvents, timeTracker) + } else super.writeAsync(aggregateEvents, clock) - private[eventuate] override def readTimeTrackerAsync: Future[TimeTracker] = + private[eventuate] override def readClockAsync: Future[Clock] = if (failureSpec.failOnSequenceNrRead && !readSequenceNumberFailed) { readSequenceNumberFailed = true Future.failed(boom) - } else super.readTimeTrackerAsync + } else super.readClockAsync } } diff --git a/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogSpec.scala index e431e726..2c289a83 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/log/EventLogSpec.scala @@ -16,8 +16,6 @@ package com.rbmhtechnology.eventuate.log -import com.rbmhtechnology.eventuate.ReplicationFilter.NoFilter - import scala.collection.immutable.Seq import akka.actor._ @@ -26,7 +24,9 @@ import akka.testkit.{TestProbe, TestKit} import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.DurableEvent._ import com.rbmhtechnology.eventuate.EventsourcingProtocol._ +import com.rbmhtechnology.eventuate.ReplicationFilter.NoFilter import com.rbmhtechnology.eventuate.ReplicationProtocol._ +import com.rbmhtechnology.eventuate.log.EventLog.Clock import com.rbmhtechnology.eventuate.log.EventLogLifecycleCassandra.TestFailureSpec import com.rbmhtechnology.eventuate.log.cassandra._ @@ -51,7 +51,7 @@ object EventLogSpec { |eventuate.log.leveldb.index-update-limit = 6 |eventuate.log.cassandra.default-port = 9142 |eventuate.log.cassandra.index-update-limit = 3 - |eventuate.log.cassandra.init-retry-backoff = 1s + |eventuate.log.cassandra.init-retry-delay = 1s """.stripMargin) val emitterIdA = "A" @@ -116,8 +116,8 @@ trait EventLogSpecSupport extends WordSpecLike with Matchers with BeforeAndAfter } def currentSequenceNr: Long = { - log.tell(GetTimeTracker, requestorProbe.ref) - requestorProbe.expectMsgClass(classOf[GetTimeTrackerSuccess]).tracker.sequenceNr + log.tell(GetClock, requestorProbe.ref) + requestorProbe.expectMsgClass(classOf[GetClockSuccess]).clock.sequenceNr } def expectedEmittedEvents(events: Seq[DurableEvent], offset: Long = 0): Seq[DurableEvent] = @@ -509,11 +509,11 @@ abstract class EventLogSpec extends TestKit(ActorSystem("test", EventLogSpec.con } "recover the current sequence number on (re)start" in { generateEmittedEvents() - log.tell(GetTimeTracker, requestorProbe.ref) - requestorProbe.expectMsgType[GetTimeTrackerSuccess].tracker.sequenceNr should be(3L) + log.tell(GetClock, requestorProbe.ref) + requestorProbe.expectMsgType[GetClockSuccess].clock.sequenceNr should be(3L) log ! "boom" - log.tell(GetTimeTracker, requestorProbe.ref) - requestorProbe.expectMsgType[GetTimeTrackerSuccess].tracker.sequenceNr should be(3L) + log.tell(GetClock, requestorProbe.ref) + requestorProbe.expectMsgType[GetClockSuccess].clock.sequenceNr should be(3L) } "recover the replication progress on (re)start" in { log.tell(SetReplicationProgress("x", 17), requestorProbe.ref) @@ -600,7 +600,7 @@ class EventLogSpecCassandra extends EventLogSpec with EventLogLifecycleCassandra super.beforeEach() probe = TestProbe() - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 0L), 0)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 0L), 0)) } def expectReplay(aggregateId: Option[String], payloads: String *): Unit = @@ -633,37 +633,37 @@ class EventLogSpecCassandra extends EventLogSpec with EventLogLifecycleCassandra "run an index update on initialization" in { writeEmittedEvents(List(event("a"), event("b"))) log ! "boom" - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 2L, vectorTime = timestamp(2L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 2L, versionVector = timestamp(2L)), 1)) } "retry an index update on initialization if sequence number read fails" in { val failureLog = createLog(TestFailureSpec(failOnSequenceNrRead = true), indexProbe.ref) - indexProbe.expectMsg(ReadTimeTrackerFailure(boom)) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 0L, vectorTime = VectorTime()), 0)) + indexProbe.expectMsg(ReadClockFailure(boom)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 0L, versionVector = VectorTime()), 0)) } "retry an index update on initialization if index update fails" in { val failureLog = createLog(TestFailureSpec(failBeforeIndexIncrementWrite = true), indexProbe.ref) indexProbe.expectMsg(UpdateIndexFailure(boom)) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 0L, vectorTime = VectorTime()), 0)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 0L, versionVector = VectorTime()), 0)) writeEmittedEvents(List(event("a"), event("b")), failureLog) failureLog ! "boom" indexProbe.expectMsg(UpdateIndexFailure(boom)) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 2L, vectorTime = timestamp(2L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 2L, versionVector = timestamp(2L)), 1)) } "run an index update after reaching the update limit with a single event batch" in { writeEmittedEvents(List(event("a"), event("b"), event("c"), event("d"))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 4L, vectorTime = timestamp(4L)), 2)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 4L, versionVector = timestamp(4L)), 2)) } "run an index update after reaching the update limit with a several event batches" in { writeEmittedEvents(List(event("a"), event("b"))) writeEmittedEvents(List(event("c"), event("d"))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 4L, vectorTime = timestamp(4L)), 2)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 4L, versionVector = timestamp(4L)), 2)) } "run an index update on initialization and after reaching the update limit" in { writeEmittedEvents(List(event("a"), event("b"))) log ! "boom" - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 2L, vectorTime = timestamp(2L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 2L, versionVector = timestamp(2L)), 1)) writeEmittedEvents(List(event("d"), event("e"), event("f"))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 5L, vectorTime = timestamp(5L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 5L, versionVector = timestamp(5L)), 1)) } "return the initial value for a replication progress" in { log.tell(GetReplicationProgress(remoteLogId), probe.ref) @@ -726,7 +726,7 @@ class EventLogSpecCassandra extends EventLogSpec with EventLogLifecycleCassandra event("b", Some("a1")), event("c", Some("a1")))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 3L, vectorTime = timestamp(3L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 3L, versionVector = timestamp(3L)), 1)) writeEmittedEvents(List( event("d", Some("a1")))) @@ -739,7 +739,7 @@ class EventLogSpecCassandra extends EventLogSpec with EventLogLifecycleCassandra event("b", Some("a1")), event("c", Some("a1")))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 3L, vectorTime = timestamp(3L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 3L, versionVector = timestamp(3L)), 1)) writeEmittedEvents(List( event("d", Some("a1")))) @@ -752,7 +752,7 @@ class EventLogSpecCassandra extends EventLogSpec with EventLogLifecycleCassandra event("b", Some("a1")), event("c", Some("a1")))) - indexProbe.expectMsg(UpdateIndexSuccess(TimeTracker(sequenceNr = 3L, vectorTime = timestamp(3L)), 1)) + indexProbe.expectMsg(UpdateIndexSuccess(Clock(sequenceNr = 3L, versionVector = timestamp(3L)), 1)) writeEmittedEvents(List( event("d", Some("a1")))) diff --git a/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationFilterSerializerSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationFilterSerializerSpec.scala index 60196cb6..efb5ca14 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationFilterSerializerSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationFilterSerializerSpec.scala @@ -18,6 +18,7 @@ package com.rbmhtechnology.eventuate.serializer import akka.actor._ import akka.serialization.Serializer + import com.rbmhtechnology.eventuate.ReplicationFilter.AndFilter import com.rbmhtechnology.eventuate.ReplicationFilter.NoFilter import com.rbmhtechnology.eventuate.ReplicationFilter.OrFilter diff --git a/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializerSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializerSpec.scala index c145014d..c1d5133a 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializerSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializerSpec.scala @@ -17,7 +17,6 @@ package com.rbmhtechnology.eventuate.serializer import akka.actor.ActorRef -import akka.testkit.TestProbe import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.ReplicationProtocol._ diff --git a/src/it/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializerSpec.scala b/src/it/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializerSpec.scala index 738c2317..9cd5e610 100644 --- a/src/it/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializerSpec.scala +++ b/src/it/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializerSpec.scala @@ -22,7 +22,7 @@ import akka.testkit.TestProbe import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.ConfirmedDelivery.DeliveryAttempt -import com.rbmhtechnology.eventuate.log.TimeTracker +import com.rbmhtechnology.eventuate.log.EventLog.Clock import com.rbmhtechnology.eventuate.serializer.DurableEventSerializerSpec.{event, ExamplePayload} import org.scalatest._ @@ -32,8 +32,8 @@ object SnapshotSerializerSpec { def nodeTuples = tree.nodes.map { node => (node.versioned, node.rejected) } } - val tracker = - TimeTracker(sequenceNr = 17L, vectorTime = VectorTime("A" -> 77L)) + val clock = + Clock(sequenceNr = 17L, versionVector = VectorTime("A" -> 77L)) def last(payload: Any) = event.copy(payload = payload) @@ -119,13 +119,13 @@ class SnapshotSerializerSpec extends WordSpec with Matchers with BeforeAndAfterA actual.nodeTuples should be(expected.nodeTuples) } - "support time tracker serialization" in { + "support event log clock serialization" in { val serialization = SerializationExtension(system1) - val initial = tracker + val initial = clock val expected = initial - serialization.deserialize(serialization.serialize(initial).get, classOf[TimeTracker]).get should be(expected) + serialization.deserialize(serialization.serialize(initial).get, classOf[Clock]).get should be(expected) } } } diff --git a/src/main/java/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolFormats.java b/src/main/java/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolFormats.java index fef04171..de062afe 100644 --- a/src/main/java/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolFormats.java +++ b/src/main/java/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolFormats.java @@ -2270,19 +2270,19 @@ public interface ReplicationReadFormatOrBuilder com.google.protobuf.ByteString getReplicatorBytes(); - // optional .VectorTimeFormat currentTargetVectorTime = 6; + // optional .VectorTimeFormat currentTargetVersionVector = 6; /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - boolean hasCurrentTargetVectorTime(); + boolean hasCurrentTargetVersionVector(); /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVectorTime(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVersionVector(); /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVectorTimeOrBuilder(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVersionVectorOrBuilder(); } /** * Protobuf type {@code ReplicationReadFormat} @@ -2371,12 +2371,12 @@ private ReplicationReadFormat( case 50: { com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder subBuilder = null; if (((bitField0_ & 0x00000020) == 0x00000020)) { - subBuilder = currentTargetVectorTime_.toBuilder(); + subBuilder = currentTargetVersionVector_.toBuilder(); } - currentTargetVectorTime_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); + currentTargetVersionVector_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(currentTargetVectorTime_); - currentTargetVectorTime_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(currentTargetVersionVector_); + currentTargetVersionVector_ = subBuilder.buildPartial(); } bitField0_ |= 0x00000020; break; @@ -2561,26 +2561,26 @@ public java.lang.String getReplicator() { } } - // optional .VectorTimeFormat currentTargetVectorTime = 6; - public static final int CURRENTTARGETVECTORTIME_FIELD_NUMBER = 6; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentTargetVectorTime_; + // optional .VectorTimeFormat currentTargetVersionVector = 6; + public static final int CURRENTTARGETVERSIONVECTOR_FIELD_NUMBER = 6; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentTargetVersionVector_; /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public boolean hasCurrentTargetVectorTime() { + public boolean hasCurrentTargetVersionVector() { return ((bitField0_ & 0x00000020) == 0x00000020); } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVectorTime() { - return currentTargetVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVersionVector() { + return currentTargetVersionVector_; } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVectorTimeOrBuilder() { - return currentTargetVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVersionVectorOrBuilder() { + return currentTargetVersionVector_; } private void initFields() { @@ -2589,7 +2589,7 @@ private void initFields() { filter_ = com.rbmhtechnology.eventuate.serializer.ReplicationFilterFormats.ReplicationFilterTreeFormat.getDefaultInstance(); targetLogId_ = ""; replicator_ = ""; - currentTargetVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + currentTargetVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2602,8 +2602,8 @@ public final boolean isInitialized() { return false; } } - if (hasCurrentTargetVectorTime()) { - if (!getCurrentTargetVectorTime().isInitialized()) { + if (hasCurrentTargetVersionVector()) { + if (!getCurrentTargetVersionVector().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -2631,7 +2631,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) output.writeBytes(5, getReplicatorBytes()); } if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeMessage(6, currentTargetVectorTime_); + output.writeMessage(6, currentTargetVersionVector_); } getUnknownFields().writeTo(output); } @@ -2664,7 +2664,7 @@ public int getSerializedSize() { } if (((bitField0_ & 0x00000020) == 0x00000020)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, currentTargetVectorTime_); + .computeMessageSize(6, currentTargetVersionVector_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -2775,7 +2775,7 @@ private Builder( private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getFilterFieldBuilder(); - getCurrentTargetVectorTimeFieldBuilder(); + getCurrentTargetVersionVectorFieldBuilder(); } } private static Builder create() { @@ -2798,10 +2798,10 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000008); replicator_ = ""; bitField0_ = (bitField0_ & ~0x00000010); - if (currentTargetVectorTimeBuilder_ == null) { - currentTargetVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + if (currentTargetVersionVectorBuilder_ == null) { + currentTargetVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } else { - currentTargetVectorTimeBuilder_.clear(); + currentTargetVersionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000020); return this; @@ -2859,10 +2859,10 @@ public com.rbmhtechnology.eventuate.serializer.ReplicationProtocolFormats.Replic if (((from_bitField0_ & 0x00000020) == 0x00000020)) { to_bitField0_ |= 0x00000020; } - if (currentTargetVectorTimeBuilder_ == null) { - result.currentTargetVectorTime_ = currentTargetVectorTime_; + if (currentTargetVersionVectorBuilder_ == null) { + result.currentTargetVersionVector_ = currentTargetVersionVector_; } else { - result.currentTargetVectorTime_ = currentTargetVectorTimeBuilder_.build(); + result.currentTargetVersionVector_ = currentTargetVersionVectorBuilder_.build(); } result.bitField0_ = to_bitField0_; onBuilt(); @@ -2899,8 +2899,8 @@ public Builder mergeFrom(com.rbmhtechnology.eventuate.serializer.ReplicationProt replicator_ = other.replicator_; onChanged(); } - if (other.hasCurrentTargetVectorTime()) { - mergeCurrentTargetVectorTime(other.getCurrentTargetVectorTime()); + if (other.hasCurrentTargetVersionVector()) { + mergeCurrentTargetVersionVector(other.getCurrentTargetVersionVector()); } this.mergeUnknownFields(other.getUnknownFields()); return this; @@ -2913,8 +2913,8 @@ public final boolean isInitialized() { return false; } } - if (hasCurrentTargetVectorTime()) { - if (!getCurrentTargetVectorTime().isInitialized()) { + if (hasCurrentTargetVersionVector()) { + if (!getCurrentTargetVersionVector().isInitialized()) { return false; } @@ -3272,121 +3272,121 @@ public Builder setReplicatorBytes( return this; } - // optional .VectorTimeFormat currentTargetVectorTime = 6; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentTargetVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + // optional .VectorTimeFormat currentTargetVersionVector = 6; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentTargetVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> currentTargetVectorTimeBuilder_; + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> currentTargetVersionVectorBuilder_; /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public boolean hasCurrentTargetVectorTime() { + public boolean hasCurrentTargetVersionVector() { return ((bitField0_ & 0x00000020) == 0x00000020); } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVectorTime() { - if (currentTargetVectorTimeBuilder_ == null) { - return currentTargetVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentTargetVersionVector() { + if (currentTargetVersionVectorBuilder_ == null) { + return currentTargetVersionVector_; } else { - return currentTargetVectorTimeBuilder_.getMessage(); + return currentTargetVersionVectorBuilder_.getMessage(); } } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public Builder setCurrentTargetVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (currentTargetVectorTimeBuilder_ == null) { + public Builder setCurrentTargetVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (currentTargetVersionVectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - currentTargetVectorTime_ = value; + currentTargetVersionVector_ = value; onChanged(); } else { - currentTargetVectorTimeBuilder_.setMessage(value); + currentTargetVersionVectorBuilder_.setMessage(value); } bitField0_ |= 0x00000020; return this; } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public Builder setCurrentTargetVectorTime( + public Builder setCurrentTargetVersionVector( com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder builderForValue) { - if (currentTargetVectorTimeBuilder_ == null) { - currentTargetVectorTime_ = builderForValue.build(); + if (currentTargetVersionVectorBuilder_ == null) { + currentTargetVersionVector_ = builderForValue.build(); onChanged(); } else { - currentTargetVectorTimeBuilder_.setMessage(builderForValue.build()); + currentTargetVersionVectorBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000020; return this; } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public Builder mergeCurrentTargetVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (currentTargetVectorTimeBuilder_ == null) { + public Builder mergeCurrentTargetVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (currentTargetVersionVectorBuilder_ == null) { if (((bitField0_ & 0x00000020) == 0x00000020) && - currentTargetVectorTime_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { - currentTargetVectorTime_ = - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(currentTargetVectorTime_).mergeFrom(value).buildPartial(); + currentTargetVersionVector_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { + currentTargetVersionVector_ = + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(currentTargetVersionVector_).mergeFrom(value).buildPartial(); } else { - currentTargetVectorTime_ = value; + currentTargetVersionVector_ = value; } onChanged(); } else { - currentTargetVectorTimeBuilder_.mergeFrom(value); + currentTargetVersionVectorBuilder_.mergeFrom(value); } bitField0_ |= 0x00000020; return this; } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public Builder clearCurrentTargetVectorTime() { - if (currentTargetVectorTimeBuilder_ == null) { - currentTargetVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + public Builder clearCurrentTargetVersionVector() { + if (currentTargetVersionVectorBuilder_ == null) { + currentTargetVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); onChanged(); } else { - currentTargetVectorTimeBuilder_.clear(); + currentTargetVersionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000020); return this; } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getCurrentTargetVectorTimeBuilder() { + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getCurrentTargetVersionVectorBuilder() { bitField0_ |= 0x00000020; onChanged(); - return getCurrentTargetVectorTimeFieldBuilder().getBuilder(); + return getCurrentTargetVersionVectorFieldBuilder().getBuilder(); } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVectorTimeOrBuilder() { - if (currentTargetVectorTimeBuilder_ != null) { - return currentTargetVectorTimeBuilder_.getMessageOrBuilder(); + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentTargetVersionVectorOrBuilder() { + if (currentTargetVersionVectorBuilder_ != null) { + return currentTargetVersionVectorBuilder_.getMessageOrBuilder(); } else { - return currentTargetVectorTime_; + return currentTargetVersionVector_; } } /** - * optional .VectorTimeFormat currentTargetVectorTime = 6; + * optional .VectorTimeFormat currentTargetVersionVector = 6; */ private com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> - getCurrentTargetVectorTimeFieldBuilder() { - if (currentTargetVectorTimeBuilder_ == null) { - currentTargetVectorTimeBuilder_ = new com.google.protobuf.SingleFieldBuilder< + getCurrentTargetVersionVectorFieldBuilder() { + if (currentTargetVersionVectorBuilder_ == null) { + currentTargetVersionVectorBuilder_ = new com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder>( - currentTargetVectorTime_, + currentTargetVersionVector_, getParentForChildren(), isClean()); - currentTargetVectorTime_ = null; + currentTargetVersionVector_ = null; } - return currentTargetVectorTimeBuilder_; + return currentTargetVersionVectorBuilder_; } // @@protoc_insertion_point(builder_scope:ReplicationReadFormat) @@ -3453,19 +3453,19 @@ com.rbmhtechnology.eventuate.serializer.DurableEventFormats.DurableEventFormatOr com.google.protobuf.ByteString getTargetLogIdBytes(); - // optional .VectorTimeFormat currentSourceVectorTime = 4; + // optional .VectorTimeFormat currentSourceVersionVector = 4; /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - boolean hasCurrentSourceVectorTime(); + boolean hasCurrentSourceVersionVector(); /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVectorTime(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVersionVector(); /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVectorTimeOrBuilder(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVersionVectorOrBuilder(); } /** * Protobuf type {@code ReplicationReadSuccessFormat} @@ -3539,12 +3539,12 @@ private ReplicationReadSuccessFormat( case 34: { com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder subBuilder = null; if (((bitField0_ & 0x00000004) == 0x00000004)) { - subBuilder = currentSourceVectorTime_.toBuilder(); + subBuilder = currentSourceVersionVector_.toBuilder(); } - currentSourceVectorTime_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); + currentSourceVersionVector_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(currentSourceVectorTime_); - currentSourceVectorTime_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(currentSourceVersionVector_); + currentSourceVersionVector_ = subBuilder.buildPartial(); } bitField0_ |= 0x00000004; break; @@ -3687,33 +3687,33 @@ public java.lang.String getTargetLogId() { } } - // optional .VectorTimeFormat currentSourceVectorTime = 4; - public static final int CURRENTSOURCEVECTORTIME_FIELD_NUMBER = 4; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentSourceVectorTime_; + // optional .VectorTimeFormat currentSourceVersionVector = 4; + public static final int CURRENTSOURCEVERSIONVECTOR_FIELD_NUMBER = 4; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentSourceVersionVector_; /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public boolean hasCurrentSourceVectorTime() { + public boolean hasCurrentSourceVersionVector() { return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVectorTime() { - return currentSourceVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVersionVector() { + return currentSourceVersionVector_; } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVectorTimeOrBuilder() { - return currentSourceVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVersionVectorOrBuilder() { + return currentSourceVersionVector_; } private void initFields() { events_ = java.util.Collections.emptyList(); replicationProgress_ = 0L; targetLogId_ = ""; - currentSourceVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + currentSourceVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -3726,8 +3726,8 @@ public final boolean isInitialized() { return false; } } - if (hasCurrentSourceVectorTime()) { - if (!getCurrentSourceVectorTime().isInitialized()) { + if (hasCurrentSourceVersionVector()) { + if (!getCurrentSourceVersionVector().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -3749,7 +3749,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) output.writeBytes(3, getTargetLogIdBytes()); } if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeMessage(4, currentSourceVectorTime_); + output.writeMessage(4, currentSourceVersionVector_); } getUnknownFields().writeTo(output); } @@ -3774,7 +3774,7 @@ public int getSerializedSize() { } if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, currentSourceVectorTime_); + .computeMessageSize(4, currentSourceVersionVector_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -3885,7 +3885,7 @@ private Builder( private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getEventsFieldBuilder(); - getCurrentSourceVectorTimeFieldBuilder(); + getCurrentSourceVersionVectorFieldBuilder(); } } private static Builder create() { @@ -3904,10 +3904,10 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); targetLogId_ = ""; bitField0_ = (bitField0_ & ~0x00000004); - if (currentSourceVectorTimeBuilder_ == null) { - currentSourceVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + if (currentSourceVersionVectorBuilder_ == null) { + currentSourceVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } else { - currentSourceVectorTimeBuilder_.clear(); + currentSourceVersionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000008); return this; @@ -3958,10 +3958,10 @@ public com.rbmhtechnology.eventuate.serializer.ReplicationProtocolFormats.Replic if (((from_bitField0_ & 0x00000008) == 0x00000008)) { to_bitField0_ |= 0x00000004; } - if (currentSourceVectorTimeBuilder_ == null) { - result.currentSourceVectorTime_ = currentSourceVectorTime_; + if (currentSourceVersionVectorBuilder_ == null) { + result.currentSourceVersionVector_ = currentSourceVersionVector_; } else { - result.currentSourceVectorTime_ = currentSourceVectorTimeBuilder_.build(); + result.currentSourceVersionVector_ = currentSourceVersionVectorBuilder_.build(); } result.bitField0_ = to_bitField0_; onBuilt(); @@ -4013,8 +4013,8 @@ public Builder mergeFrom(com.rbmhtechnology.eventuate.serializer.ReplicationProt targetLogId_ = other.targetLogId_; onChanged(); } - if (other.hasCurrentSourceVectorTime()) { - mergeCurrentSourceVectorTime(other.getCurrentSourceVectorTime()); + if (other.hasCurrentSourceVersionVector()) { + mergeCurrentSourceVersionVector(other.getCurrentSourceVersionVector()); } this.mergeUnknownFields(other.getUnknownFields()); return this; @@ -4027,8 +4027,8 @@ public final boolean isInitialized() { return false; } } - if (hasCurrentSourceVectorTime()) { - if (!getCurrentSourceVectorTime().isInitialized()) { + if (hasCurrentSourceVersionVector()) { + if (!getCurrentSourceVersionVector().isInitialized()) { return false; } @@ -4402,121 +4402,121 @@ public Builder setTargetLogIdBytes( return this; } - // optional .VectorTimeFormat currentSourceVectorTime = 4; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentSourceVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + // optional .VectorTimeFormat currentSourceVersionVector = 4; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat currentSourceVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> currentSourceVectorTimeBuilder_; + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> currentSourceVersionVectorBuilder_; /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public boolean hasCurrentSourceVectorTime() { + public boolean hasCurrentSourceVersionVector() { return ((bitField0_ & 0x00000008) == 0x00000008); } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVectorTime() { - if (currentSourceVectorTimeBuilder_ == null) { - return currentSourceVectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getCurrentSourceVersionVector() { + if (currentSourceVersionVectorBuilder_ == null) { + return currentSourceVersionVector_; } else { - return currentSourceVectorTimeBuilder_.getMessage(); + return currentSourceVersionVectorBuilder_.getMessage(); } } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public Builder setCurrentSourceVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (currentSourceVectorTimeBuilder_ == null) { + public Builder setCurrentSourceVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (currentSourceVersionVectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - currentSourceVectorTime_ = value; + currentSourceVersionVector_ = value; onChanged(); } else { - currentSourceVectorTimeBuilder_.setMessage(value); + currentSourceVersionVectorBuilder_.setMessage(value); } bitField0_ |= 0x00000008; return this; } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public Builder setCurrentSourceVectorTime( + public Builder setCurrentSourceVersionVector( com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder builderForValue) { - if (currentSourceVectorTimeBuilder_ == null) { - currentSourceVectorTime_ = builderForValue.build(); + if (currentSourceVersionVectorBuilder_ == null) { + currentSourceVersionVector_ = builderForValue.build(); onChanged(); } else { - currentSourceVectorTimeBuilder_.setMessage(builderForValue.build()); + currentSourceVersionVectorBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000008; return this; } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public Builder mergeCurrentSourceVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (currentSourceVectorTimeBuilder_ == null) { + public Builder mergeCurrentSourceVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (currentSourceVersionVectorBuilder_ == null) { if (((bitField0_ & 0x00000008) == 0x00000008) && - currentSourceVectorTime_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { - currentSourceVectorTime_ = - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(currentSourceVectorTime_).mergeFrom(value).buildPartial(); + currentSourceVersionVector_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { + currentSourceVersionVector_ = + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(currentSourceVersionVector_).mergeFrom(value).buildPartial(); } else { - currentSourceVectorTime_ = value; + currentSourceVersionVector_ = value; } onChanged(); } else { - currentSourceVectorTimeBuilder_.mergeFrom(value); + currentSourceVersionVectorBuilder_.mergeFrom(value); } bitField0_ |= 0x00000008; return this; } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public Builder clearCurrentSourceVectorTime() { - if (currentSourceVectorTimeBuilder_ == null) { - currentSourceVectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + public Builder clearCurrentSourceVersionVector() { + if (currentSourceVersionVectorBuilder_ == null) { + currentSourceVersionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); onChanged(); } else { - currentSourceVectorTimeBuilder_.clear(); + currentSourceVersionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000008); return this; } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getCurrentSourceVectorTimeBuilder() { + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getCurrentSourceVersionVectorBuilder() { bitField0_ |= 0x00000008; onChanged(); - return getCurrentSourceVectorTimeFieldBuilder().getBuilder(); + return getCurrentSourceVersionVectorFieldBuilder().getBuilder(); } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVectorTimeOrBuilder() { - if (currentSourceVectorTimeBuilder_ != null) { - return currentSourceVectorTimeBuilder_.getMessageOrBuilder(); + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getCurrentSourceVersionVectorOrBuilder() { + if (currentSourceVersionVectorBuilder_ != null) { + return currentSourceVersionVectorBuilder_.getMessageOrBuilder(); } else { - return currentSourceVectorTime_; + return currentSourceVersionVector_; } } /** - * optional .VectorTimeFormat currentSourceVectorTime = 4; + * optional .VectorTimeFormat currentSourceVersionVector = 4; */ private com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> - getCurrentSourceVectorTimeFieldBuilder() { - if (currentSourceVectorTimeBuilder_ == null) { - currentSourceVectorTimeBuilder_ = new com.google.protobuf.SingleFieldBuilder< + getCurrentSourceVersionVectorFieldBuilder() { + if (currentSourceVersionVectorBuilder_ == null) { + currentSourceVersionVectorBuilder_ = new com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder>( - currentSourceVectorTime_, + currentSourceVersionVector_, getParentForChildren(), isClean()); - currentSourceVectorTime_ = null; + currentSourceVersionVector_ = null; } - return currentSourceVectorTimeBuilder_; + return currentSourceVersionVectorBuilder_; } // @@protoc_insertion_point(builder_scope:ReplicationReadSuccessFormat) @@ -5533,20 +5533,20 @@ public Builder mergeFrom( "InfoSuccessFormat\022,\n\004info\030\001 \002(\0132\036.Replic" + "ationEndpointInfoFormat\"Y\n\035ReplicationRe" + "adEnvelopeFormat\022\'\n\007payload\030\001 \001(\0132\026.Repl", - "icationReadFormat\022\017\n\007logName\030\002 \001(\t\"\320\001\n\025R" + + "icationReadFormat\022\017\n\007logName\030\002 \001(\t\"\323\001\n\025R" + "eplicationReadFormat\022\026\n\016fromSequenceNr\030\001" + " \001(\003\022\024\n\014maxNumEvents\030\002 \001(\005\022,\n\006filter\030\003 \001" + "(\0132\034.ReplicationFilterTreeFormat\022\023\n\013targ" + - "etLogId\030\004 \001(\t\022\022\n\nreplicator\030\005 \001(\t\0222\n\027cur" + - "rentTargetVectorTime\030\006 \001(\0132\021.VectorTimeF" + - "ormat\"\251\001\n\034ReplicationReadSuccessFormat\022#" + - "\n\006events\030\001 \003(\0132\023.DurableEventFormat\022\033\n\023r" + - "eplicationProgress\030\002 \001(\003\022\023\n\013targetLogId\030" + - "\003 \001(\t\0222\n\027currentSourceVectorTime\030\004 \001(\0132\021", - ".VectorTimeFormat\"B\n\034ReplicationReadFail" + - "ureFormat\022\r\n\005cause\030\001 \001(\t\022\023\n\013targetLogId\030" + - "\002 \001(\t\"\026\n\024ReplicationDueFormatB+\n\'com.rbm" + - "htechnology.eventuate.serializerH\001" + "etLogId\030\004 \001(\t\022\022\n\nreplicator\030\005 \001(\t\0225\n\032cur" + + "rentTargetVersionVector\030\006 \001(\0132\021.VectorTi" + + "meFormat\"\254\001\n\034ReplicationReadSuccessForma" + + "t\022#\n\006events\030\001 \003(\0132\023.DurableEventFormat\022\033" + + "\n\023replicationProgress\030\002 \001(\003\022\023\n\013targetLog" + + "Id\030\003 \001(\t\0225\n\032currentSourceVersionVector\030\004", + " \001(\0132\021.VectorTimeFormat\"B\n\034ReplicationRe" + + "adFailureFormat\022\r\n\005cause\030\001 \001(\t\022\023\n\013target" + + "LogId\030\002 \001(\t\"\026\n\024ReplicationDueFormatB+\n\'c" + + "om.rbmhtechnology.eventuate.serializerH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -5582,13 +5582,13 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_ReplicationReadFormat_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ReplicationReadFormat_descriptor, - new java.lang.String[] { "FromSequenceNr", "MaxNumEvents", "Filter", "TargetLogId", "Replicator", "CurrentTargetVectorTime", }); + new java.lang.String[] { "FromSequenceNr", "MaxNumEvents", "Filter", "TargetLogId", "Replicator", "CurrentTargetVersionVector", }); internal_static_ReplicationReadSuccessFormat_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_ReplicationReadSuccessFormat_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ReplicationReadSuccessFormat_descriptor, - new java.lang.String[] { "Events", "ReplicationProgress", "TargetLogId", "CurrentSourceVectorTime", }); + new java.lang.String[] { "Events", "ReplicationProgress", "TargetLogId", "CurrentSourceVersionVector", }); internal_static_ReplicationReadFailureFormat_descriptor = getDescriptor().getMessageTypes().get(6); internal_static_ReplicationReadFailureFormat_fieldAccessorTable = new diff --git a/src/main/java/com/rbmhtechnology/eventuate/serializer/SnapshotFormats.java b/src/main/java/com/rbmhtechnology/eventuate/serializer/SnapshotFormats.java index 60e609c7..9c0c80db 100644 --- a/src/main/java/com/rbmhtechnology/eventuate/serializer/SnapshotFormats.java +++ b/src/main/java/com/rbmhtechnology/eventuate/serializer/SnapshotFormats.java @@ -4792,7 +4792,7 @@ public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.PayloadFormat // @@protoc_insertion_point(class_scope:VersionedFormat) } - public interface TimeTrackerFormatOrBuilder + public interface ClockFormatOrBuilder extends com.google.protobuf.MessageOrBuilder { // optional int64 sequenceNr = 1; @@ -4805,39 +4805,39 @@ public interface TimeTrackerFormatOrBuilder */ long getSequenceNr(); - // optional .VectorTimeFormat vectorTime = 2; + // optional .VectorTimeFormat versionVector = 2; /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - boolean hasVectorTime(); + boolean hasVersionVector(); /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVectorTime(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVersionVector(); /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVectorTimeOrBuilder(); + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVersionVectorOrBuilder(); } /** - * Protobuf type {@code TimeTrackerFormat} + * Protobuf type {@code ClockFormat} */ - public static final class TimeTrackerFormat extends + public static final class ClockFormat extends com.google.protobuf.GeneratedMessage - implements TimeTrackerFormatOrBuilder { - // Use TimeTrackerFormat.newBuilder() to construct. - private TimeTrackerFormat(com.google.protobuf.GeneratedMessage.Builder builder) { + implements ClockFormatOrBuilder { + // Use ClockFormat.newBuilder() to construct. + private ClockFormat(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private TimeTrackerFormat(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private ClockFormat(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - private static final TimeTrackerFormat defaultInstance; - public static TimeTrackerFormat getDefaultInstance() { + private static final ClockFormat defaultInstance; + public static ClockFormat getDefaultInstance() { return defaultInstance; } - public TimeTrackerFormat getDefaultInstanceForType() { + public ClockFormat getDefaultInstanceForType() { return defaultInstance; } @@ -4847,7 +4847,7 @@ public TimeTrackerFormat getDefaultInstanceForType() { getUnknownFields() { return this.unknownFields; } - private TimeTrackerFormat( + private ClockFormat( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4878,12 +4878,12 @@ private TimeTrackerFormat( case 18: { com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder subBuilder = null; if (((bitField0_ & 0x00000002) == 0x00000002)) { - subBuilder = vectorTime_.toBuilder(); + subBuilder = versionVector_.toBuilder(); } - vectorTime_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); + versionVector_ = input.readMessage(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(vectorTime_); - vectorTime_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(versionVector_); + versionVector_ = subBuilder.buildPartial(); } bitField0_ |= 0x00000002; break; @@ -4902,28 +4902,28 @@ private TimeTrackerFormat( } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_TimeTrackerFormat_descriptor; + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_ClockFormat_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_TimeTrackerFormat_fieldAccessorTable + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_ClockFormat_fieldAccessorTable .ensureFieldAccessorsInitialized( - com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.class, com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.Builder.class); + com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.class, com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.Builder.class); } - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public TimeTrackerFormat parsePartialFrom( + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ClockFormat parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new TimeTrackerFormat(input, extensionRegistry); + return new ClockFormat(input, extensionRegistry); } }; @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public com.google.protobuf.Parser getParserForType() { return PARSER; } @@ -4944,39 +4944,39 @@ public long getSequenceNr() { return sequenceNr_; } - // optional .VectorTimeFormat vectorTime = 2; - public static final int VECTORTIME_FIELD_NUMBER = 2; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat vectorTime_; + // optional .VectorTimeFormat versionVector = 2; + public static final int VERSIONVECTOR_FIELD_NUMBER = 2; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat versionVector_; /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public boolean hasVectorTime() { + public boolean hasVersionVector() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVectorTime() { - return vectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVersionVector() { + return versionVector_; } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVectorTimeOrBuilder() { - return vectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVersionVectorOrBuilder() { + return versionVector_; } private void initFields() { sequenceNr_ = 0L; - vectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + versionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasVectorTime()) { - if (!getVectorTime().isInitialized()) { + if (hasVersionVector()) { + if (!getVersionVector().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -4992,7 +4992,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) output.writeInt64(1, sequenceNr_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, vectorTime_); + output.writeMessage(2, versionVector_); } getUnknownFields().writeTo(output); } @@ -5009,7 +5009,7 @@ public int getSerializedSize() { } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, vectorTime_); + .computeMessageSize(2, versionVector_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -5023,53 +5023,53 @@ protected java.lang.Object writeReplace() return super.writeReplace(); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom(byte[] data) + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom(java.io.InputStream input) + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseFrom(input, extensionRegistry); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseDelimitedFrom(java.io.InputStream input) + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseDelimitedFrom(input); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseDelimitedFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseDelimitedFrom(input, extensionRegistry); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parseFrom( + public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5078,7 +5078,7 @@ public static com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTracke public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat prototype) { + public static Builder newBuilder(com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -5090,24 +5090,24 @@ protected Builder newBuilderForType( return builder; } /** - * Protobuf type {@code TimeTrackerFormat} + * Protobuf type {@code ClockFormat} */ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormatOrBuilder { + implements com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormatOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_TimeTrackerFormat_descriptor; + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_ClockFormat_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_TimeTrackerFormat_fieldAccessorTable + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_ClockFormat_fieldAccessorTable .ensureFieldAccessorsInitialized( - com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.class, com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.Builder.class); + com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.class, com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.Builder.class); } - // Construct using com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.newBuilder() + // Construct using com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -5119,7 +5119,7 @@ private Builder( } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getVectorTimeFieldBuilder(); + getVersionVectorFieldBuilder(); } } private static Builder create() { @@ -5130,10 +5130,10 @@ public Builder clear() { super.clear(); sequenceNr_ = 0L; bitField0_ = (bitField0_ & ~0x00000001); - if (vectorTimeBuilder_ == null) { - vectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + if (versionVectorBuilder_ == null) { + versionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); } else { - vectorTimeBuilder_.clear(); + versionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); return this; @@ -5145,23 +5145,23 @@ public Builder clone() { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_TimeTrackerFormat_descriptor; + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.internal_static_ClockFormat_descriptor; } - public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat getDefaultInstanceForType() { - return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.getDefaultInstance(); + public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat getDefaultInstanceForType() { + return com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.getDefaultInstance(); } - public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat build() { - com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat result = buildPartial(); + public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat build() { + com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat buildPartial() { - com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat result = new com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat(this); + public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat buildPartial() { + com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat result = new com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -5171,10 +5171,10 @@ public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - if (vectorTimeBuilder_ == null) { - result.vectorTime_ = vectorTime_; + if (versionVectorBuilder_ == null) { + result.versionVector_ = versionVector_; } else { - result.vectorTime_ = vectorTimeBuilder_.build(); + result.versionVector_ = versionVectorBuilder_.build(); } result.bitField0_ = to_bitField0_; onBuilt(); @@ -5182,29 +5182,29 @@ public com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat) { - return mergeFrom((com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat)other); + if (other instanceof com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat) { + return mergeFrom((com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat other) { - if (other == com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat.getDefaultInstance()) return this; + public Builder mergeFrom(com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat other) { + if (other == com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat.getDefaultInstance()) return this; if (other.hasSequenceNr()) { setSequenceNr(other.getSequenceNr()); } - if (other.hasVectorTime()) { - mergeVectorTime(other.getVectorTime()); + if (other.hasVersionVector()) { + mergeVersionVector(other.getVersionVector()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (hasVectorTime()) { - if (!getVectorTime().isInitialized()) { + if (hasVersionVector()) { + if (!getVersionVector().isInitialized()) { return false; } @@ -5216,11 +5216,11 @@ public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat parsedMessage = null; + com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (com.rbmhtechnology.eventuate.serializer.SnapshotFormats.TimeTrackerFormat) e.getUnfinishedMessage(); + parsedMessage = (com.rbmhtechnology.eventuate.serializer.SnapshotFormats.ClockFormat) e.getUnfinishedMessage(); throw e; } finally { if (parsedMessage != null) { @@ -5264,132 +5264,132 @@ public Builder clearSequenceNr() { return this; } - // optional .VectorTimeFormat vectorTime = 2; - private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat vectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + // optional .VectorTimeFormat versionVector = 2; + private com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat versionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> vectorTimeBuilder_; + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> versionVectorBuilder_; /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public boolean hasVectorTime() { + public boolean hasVersionVector() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVectorTime() { - if (vectorTimeBuilder_ == null) { - return vectorTime_; + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat getVersionVector() { + if (versionVectorBuilder_ == null) { + return versionVector_; } else { - return vectorTimeBuilder_.getMessage(); + return versionVectorBuilder_.getMessage(); } } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public Builder setVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (vectorTimeBuilder_ == null) { + public Builder setVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (versionVectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - vectorTime_ = value; + versionVector_ = value; onChanged(); } else { - vectorTimeBuilder_.setMessage(value); + versionVectorBuilder_.setMessage(value); } bitField0_ |= 0x00000002; return this; } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public Builder setVectorTime( + public Builder setVersionVector( com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder builderForValue) { - if (vectorTimeBuilder_ == null) { - vectorTime_ = builderForValue.build(); + if (versionVectorBuilder_ == null) { + versionVector_ = builderForValue.build(); onChanged(); } else { - vectorTimeBuilder_.setMessage(builderForValue.build()); + versionVectorBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000002; return this; } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public Builder mergeVectorTime(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { - if (vectorTimeBuilder_ == null) { + public Builder mergeVersionVector(com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat value) { + if (versionVectorBuilder_ == null) { if (((bitField0_ & 0x00000002) == 0x00000002) && - vectorTime_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { - vectorTime_ = - com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(vectorTime_).mergeFrom(value).buildPartial(); + versionVector_ != com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance()) { + versionVector_ = + com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.newBuilder(versionVector_).mergeFrom(value).buildPartial(); } else { - vectorTime_ = value; + versionVector_ = value; } onChanged(); } else { - vectorTimeBuilder_.mergeFrom(value); + versionVectorBuilder_.mergeFrom(value); } bitField0_ |= 0x00000002; return this; } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public Builder clearVectorTime() { - if (vectorTimeBuilder_ == null) { - vectorTime_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); + public Builder clearVersionVector() { + if (versionVectorBuilder_ == null) { + versionVector_ = com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.getDefaultInstance(); onChanged(); } else { - vectorTimeBuilder_.clear(); + versionVectorBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); return this; } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getVectorTimeBuilder() { + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder getVersionVectorBuilder() { bitField0_ |= 0x00000002; onChanged(); - return getVectorTimeFieldBuilder().getBuilder(); + return getVersionVectorFieldBuilder().getBuilder(); } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ - public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVectorTimeOrBuilder() { - if (vectorTimeBuilder_ != null) { - return vectorTimeBuilder_.getMessageOrBuilder(); + public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder getVersionVectorOrBuilder() { + if (versionVectorBuilder_ != null) { + return versionVectorBuilder_.getMessageOrBuilder(); } else { - return vectorTime_; + return versionVector_; } } /** - * optional .VectorTimeFormat vectorTime = 2; + * optional .VectorTimeFormat versionVector = 2; */ private com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder> - getVectorTimeFieldBuilder() { - if (vectorTimeBuilder_ == null) { - vectorTimeBuilder_ = new com.google.protobuf.SingleFieldBuilder< + getVersionVectorFieldBuilder() { + if (versionVectorBuilder_ == null) { + versionVectorBuilder_ = new com.google.protobuf.SingleFieldBuilder< com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormat.Builder, com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFormatOrBuilder>( - vectorTime_, + versionVector_, getParentForChildren(), isClean()); - vectorTime_ = null; + versionVector_ = null; } - return vectorTimeBuilder_; + return versionVectorBuilder_; } - // @@protoc_insertion_point(builder_scope:TimeTrackerFormat) + // @@protoc_insertion_point(builder_scope:ClockFormat) } static { - defaultInstance = new TimeTrackerFormat(true); + defaultInstance = new ClockFormat(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:TimeTrackerFormat) + // @@protoc_insertion_point(class_scope:ClockFormat) } private static com.google.protobuf.Descriptors.Descriptor @@ -5418,10 +5418,10 @@ public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFor com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_VersionedFormat_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_TimeTrackerFormat_descriptor; + internal_static_ClockFormat_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_TimeTrackerFormat_fieldAccessorTable; + internal_static_ClockFormat_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -5449,10 +5449,10 @@ public com.rbmhtechnology.eventuate.serializer.DurableEventFormats.VectorTimeFor "rrentVersionsTreeNodeFormat\"o\n\017Versioned" + "Format\022*\n\017updateTimestamp\030\001 \001(\0132\021.Vector" + "TimeFormat\022\017\n\007creator\030\002 \001(\t\022\037\n\007payload\030\003" + - " \001(\0132\016.PayloadFormat\"N\n\021TimeTrackerForma" + - "t\022\022\n\nsequenceNr\030\001 \001(\003\022%\n\nvectorTime\030\002 \001(", - "\0132\021.VectorTimeFormatB+\n\'com.rbmhtechnolo" + - "gy.eventuate.serializerH\001" + " \001(\0132\016.PayloadFormat\"K\n\013ClockFormat\022\022\n\ns" + + "equenceNr\030\001 \001(\003\022(\n\rversionVector\030\002 \001(\0132\021", + ".VectorTimeFormatB+\n\'com.rbmhtechnology." + + "eventuate.serializerH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -5489,12 +5489,12 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_VersionedFormat_descriptor, new java.lang.String[] { "UpdateTimestamp", "Creator", "Payload", }); - internal_static_TimeTrackerFormat_descriptor = + internal_static_ClockFormat_descriptor = getDescriptor().getMessageTypes().get(5); - internal_static_TimeTrackerFormat_fieldAccessorTable = new + internal_static_ClockFormat_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_TimeTrackerFormat_descriptor, - new java.lang.String[] { "SequenceNr", "VectorTime", }); + internal_static_ClockFormat_descriptor, + new java.lang.String[] { "SequenceNr", "VersionVector", }); return null; } }; diff --git a/src/main/protobuf/ReplicationProtocolFormats.proto b/src/main/protobuf/ReplicationProtocolFormats.proto index f9ea148f..65210a7a 100644 --- a/src/main/protobuf/ReplicationProtocolFormats.proto +++ b/src/main/protobuf/ReplicationProtocolFormats.proto @@ -48,14 +48,14 @@ message ReplicationReadFormat { optional ReplicationFilterTreeFormat filter = 3; optional string targetLogId = 4; optional string replicator = 5; - optional VectorTimeFormat currentTargetVectorTime = 6; + optional VectorTimeFormat currentTargetVersionVector = 6; } message ReplicationReadSuccessFormat { repeated DurableEventFormat events = 1; optional int64 replicationProgress = 2; optional string targetLogId = 3; - optional VectorTimeFormat currentSourceVectorTime = 4; + optional VectorTimeFormat currentSourceVersionVector = 4; } message ReplicationReadFailureFormat { diff --git a/src/main/protobuf/SnapshotFormats.proto b/src/main/protobuf/SnapshotFormats.proto index 74d15241..ca39840c 100644 --- a/src/main/protobuf/SnapshotFormats.proto +++ b/src/main/protobuf/SnapshotFormats.proto @@ -54,8 +54,8 @@ message VersionedFormat { optional PayloadFormat payload = 3; } -message TimeTrackerFormat { +message ClockFormat { optional int64 sequenceNr = 1; - optional VectorTimeFormat vectorTime = 2; + optional VectorTimeFormat versionVector = 2; } diff --git a/src/main/resources/reference.conf b/src/main/resources/reference.conf index 9d39d1fc..03de7c4e 100644 --- a/src/main/resources/reference.conf +++ b/src/main/resources/reference.conf @@ -14,7 +14,7 @@ akka { "com.rbmhtechnology.eventuate.ReplicationProtocol$Format" = eventuate-replication-protocol "com.rbmhtechnology.eventuate.ConcurrentVersionsTree" = eventuate-snapshot "com.rbmhtechnology.eventuate.Snapshot" = eventuate-snapshot - "com.rbmhtechnology.eventuate.log.TimeTracker" = eventuate-snapshot + "com.rbmhtechnology.eventuate.log.EventLog$Clock" = eventuate-snapshot "com.rbmhtechnology.eventuate.crdt.CRDTFormat" = eventuate-crdt } } @@ -134,15 +134,20 @@ eventuate { # eventuate.log.replication.batch-size-max partition-size-max = 131072 - # Retry backoff for event log initialization. Initialization requires - # reading the last indexed sequence number and indexing of not yet - # indexed log entries. - init-retry-backoff = 5s - # Minimum number of new events that must have been written before another # index update is triggered. index-update-limit = 128 + # Maximum number event log initialization retries. Initialization includes + # recovery of the current sequence number and version vector as well as + # indexing of not yet indexed log entries. + init-retry-max = 3 + + # Delay between event log initialization retries. Initialization includes + # recovery of the current sequence number and version vector as well as + # indexing of not yet indexed log entries. + init-retry-delay = 5s + # Maximum number of initial connection retries to a Cassandra cluster. initial-connect-retry-max = 3 diff --git a/src/main/scala/com/rbmhtechnology/eventuate/DurableEvent.scala b/src/main/scala/com/rbmhtechnology/eventuate/DurableEvent.scala index 86e2633b..5742ad3b 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/DurableEvent.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/DurableEvent.scala @@ -16,6 +16,8 @@ package com.rbmhtechnology.eventuate +import scala.collection.immutable.Seq + /** * Provider API. * @@ -102,4 +104,20 @@ object DurableEvent { def apply(emitterId: String): DurableEvent = apply(null, emitterId) -} \ No newline at end of file +} + +/** + * Implemented by protocol messages that contain a [[DurableEvent]] sequence. + */ +trait DurableEventBatch { + /** + * Event sequence. + */ + def events: Seq[DurableEvent] + + /** + * Event sequence size. + */ + def size: Int = events.size +} + diff --git a/src/main/scala/com/rbmhtechnology/eventuate/EventsourcingProtocol.scala b/src/main/scala/com/rbmhtechnology/eventuate/EventsourcingProtocol.scala index beb3c3bd..56bdb64b 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/EventsourcingProtocol.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/EventsourcingProtocol.scala @@ -36,7 +36,7 @@ object EventsourcingProtocol { * to the given `requestor`. In case of a successful write, events are sent within [[WriteSuccess]] * messages, otherwise within [[WriteFailure]] messages with `initiator` as message sender. */ - case class Write(events: Seq[DurableEvent], initiator: ActorRef, requestor: ActorRef, instanceId: Int) + case class Write(events: Seq[DurableEvent], initiator: ActorRef, requestor: ActorRef, instanceId: Int) extends DurableEventBatch /** * Success reply after a [[Write]]. diff --git a/src/main/scala/com/rbmhtechnology/eventuate/Recovery.scala b/src/main/scala/com/rbmhtechnology/eventuate/Recovery.scala index 6a1f0c16..9c2b73b0 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/Recovery.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/Recovery.scala @@ -19,12 +19,12 @@ package com.rbmhtechnology.eventuate import java.util.concurrent.TimeUnit import akka.actor._ -import akka.pattern.{ after, ask } +import akka.pattern.ask import akka.util.Timeout import com.rbmhtechnology.eventuate.EventsourcingProtocol._ import com.rbmhtechnology.eventuate.ReplicationProtocol._ -import com.rbmhtechnology.eventuate.log.TimeTracker +import com.rbmhtechnology.eventuate.log.EventLog.Clock import com.typesafe.config.Config import scala.collection.immutable.Seq @@ -51,9 +51,9 @@ private class RecoverySettings(config: Config) { * @param logName Common name of the linked local and remote event log. * @param localLogId Local event log id. * @param remoteLogId Remote event log id. - * @param tracker Time tracker value of the local event log at the beginning of disaster recovery. + * @param clock Local event log clock at the beginning of disaster recovery. */ -private case class RecoveryLink(logName: String, localLogId: String, remoteLogId: String, tracker: TimeTracker) +private case class RecoveryLink(logName: String, localLogId: String, remoteLogId: String, clock: Clock) /** * Provides disaster recovery primitives. @@ -65,7 +65,6 @@ private case class RecoveryLink(logName: String, localLogId: String, remoteLogId private class Recovery(endpoint: ReplicationEndpoint) { private val settings = new RecoverySettings(endpoint.system.settings.config) - import Recovery._ import settings._ import endpoint.system.dispatcher @@ -73,11 +72,11 @@ private class Recovery(endpoint: ReplicationEndpoint) { private implicit val scheduler = endpoint.system.scheduler /** - * Reads the time trackers from local event logs. + * Reads the clocks from local event logs. */ - def readTimeTrackers: Future[Map[String, TimeTracker]] = { - println(s"[recovery of ${endpoint.id}] Read time trackers from local event logs ...") - Future.sequence(endpoint.logNames.map(name => readTimeTracker(endpoint.logs(name)).map(name -> _))).map(_.toMap) + def readClocks: Future[Map[String, Clock]] = { + println(s"[recovery of ${endpoint.id}] Read clocks from local event logs ...") + Future.sequence(endpoint.logNames.map(name => readClock(endpoint.logs(name)).map(name -> _))).map(_.toMap) } /** @@ -97,27 +96,22 @@ private class Recovery(endpoint: ReplicationEndpoint) { Future.sequence(links.map(deleteSnapshots)).map(_ => ()) } - def readTimeTracker(targetLog: ActorRef): Future[TimeTracker] = - targetLog.ask(GetTimeTracker).mapTo[GetTimeTrackerSuccess].map(_.tracker) + def readClock(targetLog: ActorRef): Future[Clock] = + targetLog.ask(GetClock).mapTo[GetClockSuccess].map(_.clock) def readEndpointInfo(targetAcceptor: ActorSelection): Future[ReplicationEndpointInfo] = - retry(targetAcceptor.ask(GetReplicationEndpointInfo), remoteOperationRetryDelay, remoteOperationRetryMax).mapTo[GetReplicationEndpointInfoSuccess].map(_.info) + Retry(targetAcceptor.ask(GetReplicationEndpointInfo), remoteOperationRetryDelay, remoteOperationRetryMax).mapTo[GetReplicationEndpointInfoSuccess].map(_.info) def deleteSnapshots(link: RecoveryLink): Future[Unit] = - endpoint.logs(link.logName).ask(DeleteSnapshots(link.tracker.sequenceNr + 1L))(Timeout(snapshotDeletionTimeout)).flatMap { + endpoint.logs(link.logName).ask(DeleteSnapshots(link.clock.sequenceNr + 1L))(Timeout(snapshotDeletionTimeout)).flatMap { case DeleteSnapshotsSuccess => Future.successful(()) case DeleteSnapshotsFailure(e) => Future.failed(e) } - def recoveryLinks(endpointInfos: Set[ReplicationEndpointInfo], timeTrackers: Map[String, TimeTracker]) = for { + def recoveryLinks(endpointInfos: Set[ReplicationEndpointInfo], clocks: Map[String, Clock]) = for { endpointInfo <- endpointInfos logName <- endpoint.commonLogNames(endpointInfo) - } yield RecoveryLink(logName, endpoint.logId(logName), endpointInfo.logId(logName), timeTrackers(logName)) -} - -private object Recovery { - def retry[T](async: => Future[T], delay: FiniteDuration, retries: Int)(implicit ec: ExecutionContext, s: Scheduler): Future[T] = - async recoverWith { case _ if retries > 0 => after(delay, s)(retry(async, delay, retries - 1)(ec, s)) } + } yield RecoveryLink(logName, endpoint.logId(logName), endpointInfo.logId(logName), clocks(logName)) } /** @@ -219,11 +213,11 @@ private class RecoveryActor(endpointId: String, link: RecoveryLink) extends Acto import Acceptor._ def receive = { - case r: ReplicationRead if r.fromSequenceNr > link.tracker.sequenceNr + 1L => + case r: ReplicationRead if r.fromSequenceNr > link.clock.sequenceNr + 1L => println(s"[recovery of ${endpointId}] Trigger update of inconsistent replication progress at ${link.remoteLogId} ...") - sender() ! ReplicationReadSuccess(Seq(), link.tracker.sequenceNr, link.remoteLogId, link.tracker.vectorTime) + sender() ! ReplicationReadSuccess(Seq(), link.clock.sequenceNr, link.remoteLogId, link.clock.versionVector) case r: ReplicationRead => - sender() ! ReplicationReadSuccess(Seq(), r.fromSequenceNr - 1L, link.remoteLogId, link.tracker.vectorTime) + sender() ! ReplicationReadSuccess(Seq(), r.fromSequenceNr - 1L, link.remoteLogId, link.clock.versionVector) context.parent ! RecoveryStepCompleted(link) } } diff --git a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationConnection.scala b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationConnection.scala index be47c92e..0397ab4d 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationConnection.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationConnection.scala @@ -16,12 +16,6 @@ package com.rbmhtechnology.eventuate -import java.util.function.{ Function => JFunction } - -import scala.collection.JavaConverters._ - -import akka.actor._ - object ReplicationConnection { /** * Default name of the remote actor system to connect to. diff --git a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationEndpoint.scala b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationEndpoint.scala index ed13acc5..0f66da0d 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationEndpoint.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationEndpoint.scala @@ -206,8 +206,8 @@ class ReplicationEndpoint(val id: String, val logNames: Set[String], val logFact val phase1 = for { infos <- recovery.readEndpointInfos - trackers <- recovery.readTimeTrackers - links = recovery.recoveryLinks(infos, trackers) + clocks <- recovery.readClocks + links = recovery.recoveryLinks(infos, clocks) _ <- recovery.deleteSnapshots(links) } yield links diff --git a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationProtocol.scala b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationProtocol.scala index 7fede9e7..215718fd 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/ReplicationProtocol.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/ReplicationProtocol.scala @@ -18,7 +18,7 @@ package com.rbmhtechnology.eventuate import akka.actor._ -import com.rbmhtechnology.eventuate.log.TimeTracker +import com.rbmhtechnology.eventuate.log.EventLog.Clock import scala.collection.immutable.Seq @@ -67,14 +67,14 @@ object ReplicationProtocol { private[eventuate] case object ReplicationDue extends Format /** - * Requests the time tracker from an event log. + * Requests the clock from an event log. */ - private[eventuate] case object GetTimeTracker + private[eventuate] case object GetClock /** - * Success reply after a [[GetTimeTracker]]. + * Success reply after a [[GetClock]]. */ - private[eventuate] case class GetTimeTrackerSuccess(tracker: TimeTracker) + private[eventuate] case class GetClockSuccess(clock: Clock) /** * Requests all replication progresses from a log. @@ -131,7 +131,7 @@ object ReplicationProtocol { * Instructs a source log to read up to `maxNumEvents` starting `fromSequenceNr` * and applying the given replication `filter`. */ - case class ReplicationRead(fromSequenceNr: Long, maxNumEvents: Int, filter: ReplicationFilter, targetLogId: String, replicator: ActorRef, currentTargetVectorTime: VectorTime) extends Format + case class ReplicationRead(fromSequenceNr: Long, maxNumEvents: Int, filter: ReplicationFilter, targetLogId: String, replicator: ActorRef, currentTargetVersionVector: VectorTime) extends Format /** * Success reply after a [[ReplicationRead]]. @@ -141,7 +141,7 @@ object ReplicationProtocol { * or equal to the sequence number of the last read * event (if any). */ - case class ReplicationReadSuccess(events: Seq[DurableEvent], replicationProgress: Long, targetLogId: String, currentSourceVectorTime: VectorTime) extends Format + case class ReplicationReadSuccess(events: Seq[DurableEvent], replicationProgress: Long, targetLogId: String, currentSourceVersionVector: VectorTime) extends DurableEventBatch with Format /** * Failure reply after a [[ReplicationRead]]. @@ -162,7 +162,7 @@ object ReplicationProtocol { * Instructs a target log to write replicated `events` from the source log identified by * `sourceLogId` along with the last read position in the source log (`replicationProgress`). */ - case class ReplicationWrite(events: Seq[DurableEvent], sourceLogId: String, replicationProgress: Long, currentSourceVectorTime: VectorTime, initiator: ActorRef = null) + case class ReplicationWrite(events: Seq[DurableEvent], sourceLogId: String, replicationProgress: Long, currentSourceVersionVector: VectorTime, initiator: ActorRef = null) extends DurableEventBatch /** * Success reply after a [[ReplicationWrite]]. diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbSettings.scala b/src/main/scala/com/rbmhtechnology/eventuate/Retry.scala similarity index 52% rename from src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbSettings.scala rename to src/main/scala/com/rbmhtechnology/eventuate/Retry.scala index c8331810..9ae566e4 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbSettings.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/Retry.scala @@ -14,21 +14,15 @@ * limitations under the License. */ -package com.rbmhtechnology.eventuate.log.leveldb +package com.rbmhtechnology.eventuate -import akka.actor.ActorSystem -import akka.dispatch.MessageDispatcher +import akka.actor.Scheduler +import akka.pattern.after -private[eventuate] class LeveldbSettings(system: ActorSystem) { - implicit val readDispatcher: MessageDispatcher = - system.dispatchers.lookup("eventuate.log.leveldb.read-dispatcher") +import scala.concurrent._ +import scala.concurrent.duration.FiniteDuration - val rootDir: String = - system.settings.config.getString("eventuate.log.leveldb.dir") - - val fsync: Boolean = - system.settings.config.getBoolean("eventuate.log.leveldb.fsync") - - val stateSnapshotLimit: Int = - system.settings.config.getInt("eventuate.log.leveldb.state-snapshot-limit") +private object Retry { + def apply[T](async: => Future[T], delay: FiniteDuration, retries: Int)(implicit ec: ExecutionContext, s: Scheduler): Future[T] = + async recoverWith { case _ if retries > 0 => after(delay, s)(apply(async, delay, retries - 1)(ec, s)) } } diff --git a/src/main/scala/com/rbmhtechnology/eventuate/crdt/LWWRegister.scala b/src/main/scala/com/rbmhtechnology/eventuate/crdt/LWWRegister.scala index 592d423e..0ef63036 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/crdt/LWWRegister.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/crdt/LWWRegister.scala @@ -18,7 +18,7 @@ package com.rbmhtechnology.eventuate.crdt import akka.actor.ActorRef import akka.actor.ActorSystem -import com.rbmhtechnology.eventuate.{ Versioned, DurableEvent, VectorTime } +import com.rbmhtechnology.eventuate.{ DurableEvent, VectorTime } import scala.concurrent.Future diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/BatchingEventLog.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/BatchingLayer.scala similarity index 73% rename from src/main/scala/com/rbmhtechnology/eventuate/log/BatchingEventLog.scala rename to src/main/scala/com/rbmhtechnology/eventuate/log/BatchingLayer.scala index b3589e6f..64646a81 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/BatchingEventLog.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/BatchingLayer.scala @@ -18,13 +18,14 @@ package com.rbmhtechnology.eventuate.log import akka.actor._ -import com.rbmhtechnology.eventuate.EventsourcingProtocol +import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.EventsourcingProtocol._ -import com.rbmhtechnology.eventuate.ReplicationProtocol import com.rbmhtechnology.eventuate.ReplicationProtocol._ import com.typesafe.config.Config +import scala.collection.immutable.Seq + private[eventuate] class BatchingSettings(config: Config) { val batchSizeLimit = config.getInt("eventuate.log.batching.batch-size-limit") } @@ -42,7 +43,7 @@ private[eventuate] class BatchingSettings(config: Config) { * @param eventLogProps configuration object of the wrapped event log actor. The wrapped event log actor is * created as child actor of this wrapper. */ -class BatchingEventLog(eventLogProps: Props) extends Actor { +class BatchingLayer(eventLogProps: Props) extends Actor { val eventLog: ActorRef = context.actorOf(eventLogProps) @@ -60,19 +61,42 @@ class BatchingEventLog(eventLogProps: Props) extends Actor { } } -private class EmissionBatcher(eventLog: ActorRef) extends Actor { +private trait Batcher[A <: DurableEventBatch] extends Actor { val settings = new BatchingSettings(context.system.settings.config) - var batch: Vector[Write] = Vector.empty + var batch: Vector[A] = Vector.empty + + def eventLog: ActorRef + def writeRequest(batches: Seq[A]): Any + def writing: Receive val idle: Receive = { - case w: Write => - batch = batch :+ w + case w: DurableEventBatch => + batch = batch :+ w.asInstanceOf[A] writeBatch() context.become(writing) case cmd => eventLog forward cmd } + def receive = idle + + @annotation.tailrec + final def writeAll(): Unit = + if (writeBatch()) writeAll() + + final def writeBatch(): Boolean = if (batch.nonEmpty) { + var num = 0 + val (w, r) = batch.span { w => + num += w.size + num <= settings.batchSizeLimit || num == w.size + } + eventLog ! writeRequest(w) + batch = r + batch.nonEmpty + } else false +} + +private class EmissionBatcher(val eventLog: ActorRef) extends Batcher[Write] { val writing: Receive = { case w: Write => batch = batch :+ w @@ -88,35 +112,11 @@ private class EmissionBatcher(eventLog: ActorRef) extends Actor { eventLog forward cmd } - def receive = idle - - @annotation.tailrec - private def writeAll(): Unit = - if (writeBatch()) writeAll() - - private def writeBatch(): Boolean = if (batch.nonEmpty) { - var num = 0 - val (w, r) = batch.span { w => - num += w.events.size - num <= settings.batchSizeLimit || num == w.events.size - } - eventLog ! WriteN(w) - batch = r - batch.nonEmpty - } else false + def writeRequest(batches: Seq[Write]) = + WriteN(batches) } -private class ReplicationBatcher(eventLog: ActorRef) extends Actor { - val settings = new BatchingSettings(context.system.settings.config) - var batch: Vector[ReplicationWrite] = Vector.empty - - val idle: Receive = { - case w: ReplicationWrite => - batch = batch :+ w.copy(initiator = sender()) - writeBatch() - context.become(writing) - } - +private class ReplicationBatcher(val eventLog: ActorRef) extends Batcher[ReplicationWrite] { val writing: Receive = { case w: ReplicationWrite => batch = batch :+ w.copy(initiator = sender()) @@ -126,16 +126,6 @@ private class ReplicationBatcher(eventLog: ActorRef) extends Actor { writeBatch() } - def receive = idle - - private def writeBatch(): Boolean = if (batch.nonEmpty) { - var num = 0 - val (w, r) = batch.span { w => - num += w.events.size - num <= settings.batchSizeLimit || num == w.events.size - } - eventLog ! ReplicationWriteN(w) - batch = r - batch.nonEmpty - } else false + def writeRequest(batches: Seq[ReplicationWrite]) = + ReplicationWriteN(batches) } diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/EventLog.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/EventLog.scala new file mode 100644 index 00000000..b3a4626b --- /dev/null +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/EventLog.scala @@ -0,0 +1,524 @@ +/* + * Copyright (C) 2015 Red Bull Media House GmbH - all rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.rbmhtechnology.eventuate.log + +import java.io.Closeable + +import akka.actor._ +import akka.dispatch.MessageDispatcher + +import com.rbmhtechnology.eventuate._ +import com.rbmhtechnology.eventuate.EventsourcingProtocol._ +import com.rbmhtechnology.eventuate.ReplicationProtocol._ +import com.rbmhtechnology.eventuate.snapshot.filesystem._ + +import scala.collection.immutable.Seq +import scala.concurrent._ +import scala.concurrent.duration.FiniteDuration +import scala.util._ + +/** + * Event log settings implemented by storage providers. + */ +trait EventLogSettings { + /** + * Id of the dispatcher that is used for asynchronous read operations. + */ + def readDispatcherId: String + + /** + * Maximum number of events to store per partition. + */ + def partitionSizeMax: Long + + /** + * Maximum number of clock recovery retries. + */ + def initRetryMax: Int + + /** + * Delay between clock recovery retries. + */ + def initRetryDelay: FiniteDuration +} + +/** + * An abstract event log that handles [[EventsourcingProtocol]] and [[ReplicationProtocol]] messages. + * It defines a service provider interface (SPI) for event and index storage providers. '''Please note + * that the SPI is preliminary and likely to be changed in future releases'''. + * + * @tparam A Event iterator parameters type. + */ +abstract class EventLog[A](id: String) extends Actor with Stash with ActorLogging { + import EventLog._ + + protected object services { + /** + * Dispatcher for asynchronous read operations. + */ + implicit lazy val readDispatcher: MessageDispatcher = + context.system.dispatchers.lookup(settings.readDispatcherId) + + /** + * Scheduler of the current actor system. + */ + implicit lazy val scheduler: Scheduler = + context.system.scheduler + } + + /** + * This event log's snapshot store. + */ + private val snapshotStore: FilesystemSnapshotStore = + new FilesystemSnapshotStore(new FilesystemSnapshotStoreSettings(context.system), id) + + /** + * Channel to notify [[Replicator]]s, reading from this event log, about updates. + */ + private val channel: ActorRef = + context.actorOf(Props(new NotificationChannel(id))) + + /** + * Registry for event-sourced actors, views, writers and processors interacting with this event + * log. + */ + private var registry: SubscriberRegistry = + SubscriberRegistry() + + /** + * The clock that tracks the sequence number and version vector of this event log. The sequence + * number is the log's logical time. The version vector is the merge result of vector timestamps + * of all events that have been written to this event log. The version vector is used to exclude + * events from being written if they are in the event log's causal past (which makes replication + * writes idempotent). + */ + private var clock: Clock = + Clock() + + /** + * Version vectors of event logs from which events are replicated. Updated when a [[Replicator]] + * writes to this event log. This is only needed for optimization: events of which is known that + * they will be excluded from being written to a target log (see also `localClock`) can already + * be excluded here at this source log (for saving network bandwidth). + */ + private var replicaVersionVectors: Map[String, VectorTime] = + Map.empty[String, VectorTime].withDefaultValue(VectorTime.Zero) + + // ------------------------------------------------------------------------ + // TODO: only transfer vector time deltas to update replicaVersionVectors + // ------------------------------------------------------------------------ + + /** + * This event log's settings. + */ + def settings: EventLogSettings + + /** + * Asynchronously recovers the clock (during initialization). + */ + def recoverClock: Future[Clock] + + /** + * Called on successful clock recovery. + */ + def recoverClockSuccess(clock: Clock): Unit = () + + /** + * Called on failed clock recovery. + */ + def recoverClockFailure(cause: Throwable): Unit = () + + /** + * Asynchronously reads all stored replication progresses. + */ + def readReplicationProgresses: Future[Map[String, Long]] + + /** + * Asynchronously reads the replication progress for given source `logId`. + */ + def readReplicationProgress(logId: String): Future[Long] + + /** + * Asynchronously writes the replication `progress` for given source `logId`. + */ + def writeReplicationProgress(logId: String, progress: Long): Future[Unit] + + /** + * Creates a [[DurableEvent]] iterator from the given `parameters` object. + */ + def eventIterator(parameters: A): Iterator[DurableEvent] with Closeable + + /** + * Creates an event iterator parameters object that encodes the given parameters. + * + * @param fromSequenceNr sequence number to start iteration (inclusive). + * @param toSequenceNr sequence number to stop iteration (inclusive). + */ + def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long): A + + /** + * Creates an event iterator parameters object that encodes the given parameters. + * + * @param fromSequenceNr sequence number to start iteration (inclusive). + * @param toSequenceNr sequence number to stop iteration (inclusive). + * @param aggregateId must be contained in [[DurableEvent.destinationAggregateIds]]. + */ + def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long, aggregateId: String): A + + /** + * Asynchronously batch-reads events from the raw event log. At most `max` events must be + * returned that are within the sequence number bounds `fromSequenceNr` and `toSequenceNr` + * and that pass the given `filter`. + * + * @param fromSequenceNr sequence number to start reading (inclusive). + * @param toSequenceNr sequence number to stop reading (inclusive) + * or earlier if `max` events have already been read. + */ + def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: DurableEvent => Boolean): Future[ReadResult] + + /** + * Synchronously writes `events` to the given `partition`. The partition is calculated from + * the configured `partitionSizeMax` and the current sequence number. + * + * Because of the synchronous write requirement, it is important that the dispatcher of an + * event log is a pinned dispatcher. Asynchronous writes will be supported in future versions. + * + * @ [[EventLogSettings]] + */ + def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit + + private def initializing: Receive = { + case RecoverySuccess(c) => + clock = c + recoverClockSuccess(c) + unstashAll() + context.become(initialized) + case RecoveryFailure(e) => + log.error(e, "Cannot recover clock") + context.stop(self) + case other => + stash() + } + + private def initialized: Receive = { + case GetClock => + sender() ! GetClockSuccess(clock) + case GetReplicationProgresses => + import services.readDispatcher + val sdr = sender() + readReplicationProgresses onComplete { + case Success(p) => sdr ! GetReplicationProgressesSuccess(p) + case Failure(e) => sdr ! GetReplicationProgressesFailure(e) + } + case GetReplicationProgress(sourceLogId) => + import services.readDispatcher + val sdr = sender() + val tvv = clock.versionVector + readReplicationProgress(sourceLogId) onComplete { + case Success(p) => sdr ! GetReplicationProgressSuccess(sourceLogId, p, tvv) + case Failure(e) => sdr ! GetReplicationProgressFailure(e) + } + case SetReplicationProgress(sourceLogId, progress) => + val sdr = sender() + implicit val dispatcher = context.dispatcher + writeReplicationProgress(sourceLogId, progress) onComplete { + case Success(_) => sdr ! SetReplicationProgressSuccess(sourceLogId, progress) + case Failure(e) => sdr ! SetReplicationProgressFailure(e) + } + case Replay(fromSequenceNr, max, requestor, Some(emitterAggregateId), iid) => + val iteratorSettings = eventIteratorParameters(fromSequenceNr, clock.sequenceNr, emitterAggregateId) // avoid async evaluation + registry = registry.registerAggregateSubscriber(context.watch(requestor), emitterAggregateId) + replayer(requestor, eventIterator(iteratorSettings), fromSequenceNr) ! ReplayNext(max, iid) + case Replay(fromSequenceNr, max, requestor, None, iid) => + val iteratorSettings = eventIteratorParameters(fromSequenceNr, clock.sequenceNr) // avoid async evaluation + registry = registry.registerDefaultSubscriber(context.watch(requestor)) + replayer(requestor, eventIterator(iteratorSettings), fromSequenceNr) ! ReplayNext(max, iid) + case r @ ReplicationRead(from, max, filter, targetLogId, _, currentTargetVersionVector) => + import services.readDispatcher + val sdr = sender() + channel ! r + read(from, clock.sequenceNr, max, evt => evt.replicable(currentTargetVersionVector, filter)) onComplete { + case Success(ReadResult(events, progress)) => + val reply = ReplicationReadSuccess(events, progress, targetLogId, null) + self.tell(reply, sdr) + case Failure(cause) => + val reply = ReplicationReadFailure(cause.getMessage, targetLogId) + sdr ! reply + channel ! reply + } + case r @ ReplicationReadSuccess(events, _, targetLogId, _) => + // Post-exclude events using a possibly updated version vector received from the + // target. This is an optimization to save network bandwidth. If omitted, events + // are still excluded at target based on the current local version vector at the + // target (for correctness). + val currentTargetVersionVector = replicaVersionVectors(targetLogId) + val updated = events.filterNot(_.before(currentTargetVersionVector)) + val reply = r.copy(updated, currentSourceVersionVector = clock.versionVector) + sender() ! reply + channel ! reply + logFilterStatistics("source", events, updated) + case w: Write => + processWrites(Seq(w)) + case WriteN(writes) => + processWrites(writes) + sender() ! WriteNComplete + case w: ReplicationWrite => + processReplicationWrites(Seq(w.copy(initiator = sender()))) + case ReplicationWriteN(writes) => + processReplicationWrites(writes) + sender() ! ReplicationWriteNComplete + case LoadSnapshot(emitterId, requestor, iid) => + import services.readDispatcher + snapshotStore.loadAsync(emitterId) onComplete { + case Success(s) => requestor ! LoadSnapshotSuccess(s, iid) + case Failure(e) => requestor ! LoadSnapshotFailure(e, iid) + } + case SaveSnapshot(snapshot, initiator, requestor, iid) => + import context.dispatcher + snapshotStore.saveAsync(snapshot) onComplete { + case Success(_) => requestor.tell(SaveSnapshotSuccess(snapshot.metadata, iid), initiator) + case Failure(e) => requestor.tell(SaveSnapshotFailure(snapshot.metadata, e, iid), initiator) + } + case DeleteSnapshots(lowerSequenceNr) => + import context.dispatcher + val sdr = sender() + snapshotStore.deleteAsync(lowerSequenceNr) onComplete { + case Success(_) => sdr ! DeleteSnapshotsSuccess + case Failure(e) => sdr ! DeleteSnapshotsFailure(e) + } + case Terminated(requestor) => + registry = registry.unregisterSubscriber(requestor) + } + + override def receive = + initializing + + private[eventuate] def currentSystemTime: Long = + System.currentTimeMillis + + private def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable, fromSequenceNr: Long): ActorRef = + context.actorOf(Props(new ChunkedEventReplay(requestor, iterator)).withDispatcher(settings.readDispatcherId)) + + private def processWrites(writes: Seq[Write]): Unit = { + val result = for { + (partition, clock1) <- Try(adjustSequenceNr(writes.map(_.size).sum, settings.partitionSizeMax, clock)) + (updatedWrites, clock2) = prepareWrites(id, writes, currentSystemTime, clock1) + updatedEvents = updatedWrites.flatMap(_.events) + _ <- Try(write(updatedEvents, partition, clock2)) + } yield (updatedWrites, updatedEvents, clock2) + + result match { + case Success((updatedWrites, updatedEvents, clock2)) => + clock = clock2 + updatedWrites.foreach(w => registry.pushWriteSuccess(w.events, w.initiator, w.requestor, w.instanceId)) + channel ! NotificationChannel.Updated(updatedEvents) + case Failure(e) => + writes.foreach(w => registry.pushWriteFailure(w.events, w.initiator, w.requestor, w.instanceId, e)) + } + } + + private def processReplicationWrites(writes: Seq[ReplicationWrite]): Unit = { + writes.foreach { write => + replicaVersionVectors = replicaVersionVectors.updated(write.sourceLogId, write.currentSourceVersionVector) + } + val result = for { + (partition, clock1) <- Try(adjustSequenceNr(writes.map(_.size).sum, settings.partitionSizeMax, clock)) + (updatedWrites, clock2) = prepareReplicationWrites(id, writes, clock1) + updatedEvents = updatedWrites.flatMap(_.events) + _ <- Try(write(updatedEvents, partition, clock2)) + } yield (updatedWrites, updatedEvents, clock2) + + result match { + case Success((updatedWrites, updatedEvents, clock2)) => + clock = clock2 + updatedWrites.foreach { w => + val rws = ReplicationWriteSuccess(w.size, w.replicationProgress, clock2.versionVector) + val sdr = w.initiator + registry.pushReplicateSuccess(w.events) + channel ! rws + implicit val dispatcher = context.system.dispatchers.defaultGlobalDispatcher + writeReplicationProgress(w.sourceLogId, w.replicationProgress) onComplete { + case Success(_) => + sdr ! rws + case Failure(e) => + // Write failure of replication progress can be ignored. Using a stale + // progress to resume replication will redundantly read events from a + // source log but these events will be successfully identified as + // duplicates, either at source or latest at target. + log.warning(s"Writing of replication progress failed: ${e.getMessage}") + sdr ! ReplicationWriteFailure(e) + } + } + channel ! NotificationChannel.Updated(updatedEvents) + case Failure(e) => + writes.foreach { write => + write.initiator ! ReplicationWriteFailure(e) + } + } + } + + private def prepareWrites(logId: String, writes: Seq[Write], systemTimestamp: Long, clock: Clock): (Seq[Write], Clock) = + writes.foldLeft((Vector.empty[Write], clock)) { + case ((writes2, clock2), write) => prepareWrite(logId, write.events, systemTimestamp, clock2) match { + case (updated, clock3) => (writes2 :+ write.copy(events = updated), clock3) + } + } + + private def prepareReplicationWrites(logId: String, writes: Seq[ReplicationWrite], clock: Clock): (Seq[ReplicationWrite], Clock) = { + writes.foldLeft((Vector.empty[ReplicationWrite], clock)) { + case ((writes2, clock2), write) => prepareReplicationWrite(logId, write.events, write.replicationProgress, clock2) match { + case (updated, clock3) => (writes2 :+ write.copy(events = updated), clock3) + } + } + } + + private def prepareWrite(logId: String, events: Seq[DurableEvent], systemTimestamp: Long, clock: Clock): (Seq[DurableEvent], Clock) = { + var snr = clock.sequenceNr + var lvv = clock.versionVector + + val updated = events.map { e => + snr += 1L + + val e2 = e.prepare(logId, snr, systemTimestamp) + lvv = lvv.merge(e2.vectorTimestamp) + e2 + } + + (updated, clock.copy(sequenceNr = snr, versionVector = lvv)) + } + + private def prepareReplicationWrite(logId: String, events: Seq[DurableEvent], replicationProgress: Long, clock: Clock): (Seq[DurableEvent], Clock) = { + var snr = clock.sequenceNr + var lvv = clock.versionVector + + val updated = events.foldLeft(Vector.empty[DurableEvent]) { + case (acc, e) if e.before(clock.versionVector) => + // Exclude events from writing that are in the causal past of this event log. Excluding + // them at the target is needed for correctness. Events are also excluded at sources + // (to save network bandwidth) but this is only an optimization which cannot achieve + // 100% filtering coverage for certain replication network topologies. + acc + case (acc, e) => + snr += 1L + + val e2 = e.prepare(logId, snr, e.systemTimestamp) + lvv = lvv.merge(e2.vectorTimestamp) + acc :+ e2 + } + logFilterStatistics("target", events, updated) + (updated, clock.copy(sequenceNr = snr, versionVector = lvv)) + } + + private def logFilterStatistics(location: String, before: Seq[DurableEvent], after: Seq[DurableEvent]): Unit = { + val bl = before.length + val al = after.length + if (al < bl) { + val diff = bl - al + val perc = diff * 100.0 / bl + log.info(f"[$id] excluded $diff events ($perc%3.1f%% at $location)") + } + } + + override def preStart(): Unit = { + import services._ + Retry(recoverClock, settings.initRetryDelay, settings.initRetryMax) onComplete { + case Success(c) => self ! RecoverySuccess(c) + case Failure(e) => self ! RecoveryFailure(e) + } + } +} + +object EventLog { + /** + * A clock that tracks the current sequence number and the version vector of an event log. + * The version vector is the merge result of vector timestamps of all events that have been + * written to an event log. + */ + case class Clock(sequenceNr: Long = 0L, versionVector: VectorTime = VectorTime.Zero) { + /** + * Advances `sequenceNr` by given `delta`. + */ + def advanceSequenceNr(delta: Long = 1L): Clock = + copy(sequenceNr = sequenceNr + delta) + + /** + * Sets `sequenceNr` to the event's local sequence number and merges `versionVector` with + * the event's vector timestamp. + */ + def update(event: DurableEvent): Clock = + copy(sequenceNr = event.localSequenceNr, versionVector = versionVector.merge(event.vectorTimestamp)) + } + + /** + * Result of an asynchronous batch read operation. + * + * @param events Read events. + * @param to Last read position. + */ + case class ReadResult(events: Seq[DurableEvent], to: Long) + + /** + * Internally sent to an [[EventLog]] after successful clock recovery. + */ + private case class RecoverySuccess(clock: Clock) + + /** + * Internally sent to an [[EventLog]] after failed clock recovery. + */ + private case class RecoveryFailure(cause: Throwable) + + /** + * Partition number for given `sequenceNr`. + */ + def partitionOf(sequenceNr: Long, partitionSizeMax: Long): Long = + if (sequenceNr == 0L) -1L else (sequenceNr - 1L) / partitionSizeMax + + /** + * Remaining partition size given the current `sequenceNr`. + */ + def remainingPartitionSize(sequenceNr: Long, partitionSizeMax: Long): Long = { + val m = sequenceNr % partitionSizeMax + if (m == 0L) m else partitionSizeMax - m + } + + /** + * First sequence number of given `partition`. + */ + def firstSequenceNr(partition: Long, partitionSizeMax: Long): Long = + partition * partitionSizeMax + 1L + + /** + * Last sequence number of given `partition`. + */ + def lastSequenceNr(partition: Long, partitionSizeMax: Long): Long = + (partition + 1L) * partitionSizeMax + + /** + * Adjusts `clock.sequenceNumber` if a batch of `batchSize` doesn't fit in the current partition. + */ + private def adjustSequenceNr(batchSize: Long, maxBatchSize: Long, clock: Clock): (Long, Clock) = { + require(batchSize <= maxBatchSize, s"write batch size (${batchSize}) must not be greater than maximum partition size (${maxBatchSize})") + + val currentPartition = partitionOf(clock.sequenceNr, maxBatchSize) + val remainingSize = remainingPartitionSize(clock.sequenceNr, maxBatchSize) + if (remainingSize < batchSize) { + (currentPartition + 1L, clock.advanceSequenceNr(remainingSize)) + } else { + (currentPartition, clock) + } + } +} diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/NotificationChannel.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/NotificationChannel.scala index 0a236b3e..eff2a0a4 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/NotificationChannel.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/NotificationChannel.scala @@ -43,7 +43,7 @@ private class NotificationChannel(logId: String) extends Actor { case Updated(events) => registry.foreach { case (targetLogId, reg) => - if (!reading.contains(targetLogId) && events.exists(_.replicable(reg.currentTargetVectorTime, reg.filter))) { + if (!reading.contains(targetLogId) && events.exists(_.replicable(reg.currentTargetVersionVector, reg.filter))) { reg.replicator ! ReplicationDue } } @@ -56,7 +56,7 @@ private class NotificationChannel(logId: String) extends Actor { reading -= r.targetLogId case w: ReplicationWrite => registry.get(w.sourceLogId) match { - case Some(reg) => registry += (w.sourceLogId -> reg.copy(currentTargetVectorTime = w.currentSourceVectorTime)) + case Some(reg) => registry += (w.sourceLogId -> reg.copy(currentTargetVersionVector = w.currentSourceVersionVector)) case None => } } diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/TimeTracker.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/TimeTracker.scala deleted file mode 100644 index 06c42b25..00000000 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/TimeTracker.scala +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Copyright (C) 2015 Red Bull Media House GmbH - all rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.rbmhtechnology.eventuate.log - -import akka.event.LoggingAdapter - -import com.rbmhtechnology.eventuate._ -import com.rbmhtechnology.eventuate.EventsourcingProtocol.Write -import com.rbmhtechnology.eventuate.ReplicationProtocol.ReplicationWrite - -import scala.collection.immutable.Seq - -/** - * Tracks the current sequence number and vector time of an event log. Besides maintaining the - * current sequence number of an event log it also maintains the merged vector times of logged - * events. The merged vector time is used to select those events for replication that are not - * in the causal past of an event log. - * - * Please note that the time tracker is not a vector clock. Vector clocks are maintained by event - * sourced actors directly (see also [[VectorClock]]). - * - * @param updateCount Number of updates made to the log. - * @param sequenceNr Current sequence number of the log. - * @param vectorTime Current vector time of the log. This is the merge - * result of all event vector timestamps in the log. - */ -private[eventuate] case class TimeTracker(updateCount: Long = 0L, sequenceNr: Long = 0L, vectorTime: VectorTime = VectorTime.Zero) { - def advanceUpdateCount(): Unit = - copy(updateCount = updateCount + 1L) - - def advanceSequenceNr(delta: Long = 1L): TimeTracker = - copy(sequenceNr = sequenceNr + delta) - - def update(event: DurableEvent): TimeTracker = - copy(sequenceNr = event.localSequenceNr, vectorTime = vectorTime.merge(event.vectorTimestamp)) - - def prepareWrites(logId: String, writes: Seq[Write], systemTimestamp: Long): (Seq[Write], TimeTracker) = - writes.foldLeft((Vector.empty[Write], this)) { - case ((writes2, tracker2), write) => tracker2.prepareWrite(logId, write.events, systemTimestamp) match { - case (updated, tracker3) => (writes2 :+ write.copy(events = updated), tracker3) - } - } - - def prepareReplicates(logId: String, writes: Seq[ReplicationWrite], logger: LoggingAdapter): (Seq[ReplicationWrite], TimeTracker) = { - writes.foldLeft((Vector.empty[ReplicationWrite], this)) { - case ((writes2, tracker2), write) => tracker2.prepareReplicate(logId, write.events, write.replicationProgress, logger) match { - case (updated, tracker3) => (writes2 :+ write.copy(events = updated), tracker3) - } - } - } - - def prepareWrite(logId: String, events: Seq[DurableEvent], systemTimestamp: Long): (Seq[DurableEvent], TimeTracker) = { - var snr = sequenceNr - var upd = updateCount - var lvt = vectorTime - - val updated = events.map { e => - snr += 1L - upd += 1L - - val e2 = e.prepare(logId, snr, systemTimestamp) - lvt = lvt.merge(e2.vectorTimestamp) - e2 - } - - (updated, copy(updateCount = upd, sequenceNr = snr, vectorTime = lvt)) - } - - def prepareReplicate(logId: String, events: Seq[DurableEvent], replicationProgress: Long, logger: LoggingAdapter): (Seq[DurableEvent], TimeTracker) = { - var snr = sequenceNr - var upd = updateCount - var lvt = vectorTime - - val updated = events.foldLeft(Vector.empty[DurableEvent]) { - case (acc, e) if e.before(vectorTime) => - // Exclude events from writing that are in the causal past of this event log. - // Excluding them at the target is needed for correctness. Events are also - // filtered at sources (to reduce network bandwidth usage) but this is only - // an optimization which cannot achieve 100% filtering coverage for certain - // replication network topologies. - acc - case (acc, e) => - snr += 1L - upd += 1L - - val e2 = e.prepare(logId, snr, e.systemTimestamp) - lvt = lvt.merge(e2.vectorTimestamp) - acc :+ e2 - } - TimeTracker.logFilterStatistics(logId, "target", events, updated, logger) - (updated, copy(updateCount = upd, sequenceNr = snr, vectorTime = lvt)) - } -} - -private[eventuate] object TimeTracker { - def logFilterStatistics(logId: String, location: String, before: Seq[DurableEvent], after: Seq[DurableEvent], logger: LoggingAdapter): Unit = { - val bl = before.length - val al = after.length - if (al < bl) { - val diff = bl - al - val perc = diff * 100.0 / bl - logger.info(f"[$logId] excluded $diff events ($perc%3.1f%% at $location)") - } - } -} \ No newline at end of file diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/Cassandra.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/Cassandra.scala index 8961aedd..dedd2697 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/Cassandra.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/Cassandra.scala @@ -27,7 +27,7 @@ import com.datastax.driver.core.exceptions.NoHostAvailableException import com.datastax.driver.core.utils.Bytes import com.rbmhtechnology.eventuate.DurableEvent -import com.rbmhtechnology.eventuate.log.TimeTracker +import com.rbmhtechnology.eventuate.log.EventLog.Clock import scala.concurrent.Future import scala.util._ @@ -103,7 +103,7 @@ class Cassandra(val system: ExtendedActorSystem) extends Extension { extension = * Settings used by the Cassandra storage backend. Closed when the `ActorSystem` of * this extension terminates. */ - private[eventuate] val settings = new CassandraSettings(system.settings.config) + private[eventuate] val settings = new CassandraEventLogSettings(system.settings.config) /** * Serializer used by the Cassandra storage backend. Closed when the `ActorSystem` of @@ -112,9 +112,9 @@ class Cassandra(val system: ExtendedActorSystem) extends Extension { extension = private[eventuate] val serializer = SerializationExtension(system) private val logging = Logging(system, this) - private val statements = new CassandraEventStatements with CassandraAggregateEventStatements with CassandraTimeTrackerStatements with CassandraReplicationProgressStatements { + private val statements = new CassandraEventStatements with CassandraAggregateEventStatements with CassandraClockStatements with CassandraReplicationProgressStatements { - override def settings: CassandraSettings = + override def settings: CassandraEventLogSettings = extension.settings } @@ -129,7 +129,7 @@ class Cassandra(val system: ExtendedActorSystem) extends Extension { extension = if (keyspaceAutoCreate) _session.execute(createKeySpaceStatement) - _session.execute(createTimeTrackerTableStatement) + _session.execute(createClockTableStatement) _session.execute(createReplicationProgressTableStatement) } match { case Success(_) => logging.info("Cassandra extension initialized") @@ -193,11 +193,11 @@ class Cassandra(val system: ExtendedActorSystem) extends Extension { extension = private[eventuate] def prepareReadAggregateEvents(logId: String): PreparedStatement = session.prepare(readAggregateEventsStatement(logId)).setConsistencyLevel(readConsistency) - private[eventuate] val preparedWriteTimeTrackerStatement: PreparedStatement = - session.prepare(writeTimeTrackerStatement).setConsistencyLevel(writeConsistency) + private[eventuate] val preparedWriteClockStatement: PreparedStatement = + session.prepare(writeClockStatement).setConsistencyLevel(writeConsistency) - private[eventuate] val preparedReadTimeTrackerStatement: PreparedStatement = - session.prepare(readTimeTrackerStatement).setConsistencyLevel(readConsistency) + private[eventuate] val preparedReadClockStatement: PreparedStatement = + session.prepare(readClockStatement).setConsistencyLevel(readConsistency) private[eventuate] val preparedWriteReplicationProgressStatement: PreparedStatement = session.prepare(writeReplicationProgressStatement).setConsistencyLevel(writeConsistency) @@ -214,11 +214,11 @@ class Cassandra(val system: ExtendedActorSystem) extends Extension { extension = private[eventuate] def eventFromByteBuffer(buffer: ByteBuffer): DurableEvent = serializer.deserialize(Bytes.getArray(buffer), classOf[DurableEvent]).get - private[eventuate] def timeTrackerToByteBuffer(tracker: TimeTracker): ByteBuffer = - ByteBuffer.wrap(serializer.serialize(tracker).get) + private[eventuate] def clockToByteBuffer(clock: Clock): ByteBuffer = + ByteBuffer.wrap(serializer.serialize(clock).get) - private[eventuate] def timeTrackerFromByteBuffer(buffer: ByteBuffer): TimeTracker = - serializer.deserialize(Bytes.getArray(buffer), classOf[TimeTracker]).get + private[eventuate] def clockFromByteBuffer(buffer: ByteBuffer): Clock = + serializer.deserialize(Bytes.getArray(buffer), classOf[Clock]).get private[eventuate] def executeBatch(body: BatchStatement => Unit): Unit = session.execute(withBatch(body)) diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLog.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLog.scala index c8dff9d1..4fbae332 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLog.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLog.scala @@ -17,20 +17,21 @@ package com.rbmhtechnology.eventuate.log.cassandra import java.io.Closeable -import java.lang.{ Long => JLong } +import java.util.concurrent.TimeUnit -import akka.actor._ +import akka.actor.Props +import akka.pattern.ask +import akka.util.Timeout import com.rbmhtechnology.eventuate._ -import com.rbmhtechnology.eventuate.EventsourcingProtocol._ -import com.rbmhtechnology.eventuate.ReplicationProtocol._ import com.rbmhtechnology.eventuate.log._ -import com.rbmhtechnology.eventuate.log.TimeTracker._ -import com.rbmhtechnology.eventuate.snapshot.filesystem._ +import com.rbmhtechnology.eventuate.log.EventLog._ import scala.collection.immutable.Seq -import scala.language.implicitConversions -import scala.util._ +import scala.concurrent.Future +import scala.language.postfixOps + +case class CassandraEventIteratorParameters(fromSequenceNr: Long, indexSequenceNr: Long, toSequenceNr: Long, aggregateId: Option[String]) /** * An event log actor with [[http://cassandra.apache.org/ Apache Cassandra]] as storage backend. It uses @@ -64,255 +65,138 @@ import scala.util._ * @see [[Cassandra]] * @see [[DurableEvent]] */ -class CassandraEventLog(val id: String) extends Actor with Stash with ActorLogging { +class CassandraEventLog(id: String) extends EventLog[CassandraEventIteratorParameters](id) { import CassandraEventLog._ import CassandraIndex._ - import NotificationChannel._ if (!isValidEventLogId(id)) throw new IllegalArgumentException(s"invalid id '$id' specified - Cassandra allows alphanumeric and underscore characters only") - val eventStream = context.system.eventStream - val cassandra: Cassandra = Cassandra(context.system) + override val settings: EventLogSettings = new CassandraEventLogSettings(context.system.settings.config) + private val cassandra: Cassandra = Cassandra(context.system) cassandra.createEventTable(id) cassandra.createAggregateEventTable(id) - private val preparedWriteEventStatement = cassandra.prepareWriteEvent(id) - - private val notificationChannel = context.actorOf(Props(new NotificationChannel(id))) - private val snapshotStore = new FilesystemSnapshotStore(new FilesystemSnapshotStoreSettings(context.system), id) private val progressStore = createReplicationProgressStore(cassandra, id) - private val eventReader = createEventReader(cassandra, id) - private val index = createIndex(cassandra, eventReader, id) - - private var registry = SubscriberRegistry() - private var timeTracker = TimeTracker() - private var timeCache = Map.empty[String, VectorTime].withDefaultValue(VectorTime.Zero) - - // ------------------------------------------------------ - // TODO: consider exchanging only vector time deltas - // - // Messages: - // - // - ReplicationRead - // - ReplicationReadSuccess - // - ReplicationWrite - // - ReplicationWriteSuccess - // - // This optimization might be necessary if many event- - // sourced actors use their own entry in vector clocks. - // ------------------------------------------------------ - - def initializing: Receive = { - case Initialize(t) => - timeTracker = t - unstashAll() - context.become(initialized) - case other => - stash() - } + private val eventLogStore = createEventLogStore(cassandra, id) + private val indexStore = createIndexStore(cassandra, id) + private val index = createIndex(cassandra, indexStore, eventLogStore, id) - def initialized: Receive = { - case GetTimeTracker => - sender() ! GetTimeTrackerSuccess(timeTracker) - case GetReplicationProgresses => - import cassandra.readDispatcher - val sdr = sender() - progressStore.readReplicationProgressesAsync onComplete { - case Success(p) => sdr ! GetReplicationProgressesSuccess(p) - case Failure(e) => sdr ! GetReplicationProgressesFailure(e) - } - case GetReplicationProgress(sourceLogId) => - import cassandra.readDispatcher - val sdr = sender() - val tvt = timeTracker.vectorTime - progressStore.readReplicationProgressAsync(sourceLogId) onComplete { - case Success(p) => sdr ! GetReplicationProgressSuccess(sourceLogId, p, tvt) - case Failure(e) => sdr ! GetReplicationProgressFailure(e) - } - case SetReplicationProgress(sourceLogId, progress) => - val sdr = sender() - implicit val dispatcher = context.system.dispatchers.defaultGlobalDispatcher - progressStore.writeReplicationProgressAsync(sourceLogId, progress) onComplete { - case Success(_) => sdr ! SetReplicationProgressSuccess(sourceLogId, progress) - case Failure(e) => sdr ! SetReplicationProgressFailure(e) - } - case r @ Replay(fromSequenceNr, max, requestor, Some(emitterAggregateId), iid) => - registry = registry.registerAggregateSubscriber(context.watch(requestor), emitterAggregateId) - index forward ReplayIndex(fromSequenceNr, timeTracker.sequenceNr, max, requestor, emitterAggregateId, iid) - case Replay(fromSequenceNr, max, requestor, None, iid) => - val toSequenceNr = timeTracker.sequenceNr // avoid async evaluation - registry = registry.registerDefaultSubscriber(context.watch(requestor)) - replayer(requestor, eventReader.eventIterator(fromSequenceNr, toSequenceNr), fromSequenceNr) ! ReplayNext(max, iid) - case r @ ReplicationRead(from, max, filter, targetLogId, _, currentTargetVectorTime) => - import cassandra.readDispatcher - val sdr = sender() - notificationChannel ! r - eventReader.readAsync(from, timeTracker.sequenceNr, max, filter, currentTargetVectorTime, targetLogId) onComplete { - case Success(result) => - val reply = ReplicationReadSuccess(result.events, result.to, targetLogId, null) - self.tell(reply, sdr) - case Failure(cause) => - val reply = ReplicationReadFailure(cause.getMessage, targetLogId) - sdr ! reply - notificationChannel ! reply - } - case r @ ReplicationReadSuccess(events, _, targetLogId, _) => - // Post-filter events using a possibly updated vector time received from the target. - // This is an optimization to reduce network bandwidth usage. If omitted, events are - // still filtered at target based on the current local vector time at the target (for - // correctness). - val currentTargetVectorTime = timeCache(targetLogId) - val updated = events.filterNot(_.before(currentTargetVectorTime)) - val reply = r.copy(updated, currentSourceVectorTime = timeTracker.vectorTime) - sender() ! reply - notificationChannel ! reply - logFilterStatistics(id, "source", events, updated, log) - case w: Write => - writeN(Seq(w)) - case WriteN(writes) => - writeN(writes) - sender() ! WriteNComplete - case w: ReplicationWrite => - replicateN(Seq(w.copy(initiator = sender()))) - case ReplicationWriteN(writes) => - replicateN(writes) - sender() ! ReplicationWriteNComplete - case LoadSnapshot(emitterId, requestor, iid) => - import cassandra.readDispatcher - snapshotStore.loadAsync(emitterId) onComplete { - case Success(s) => requestor ! LoadSnapshotSuccess(s, iid) - case Failure(e) => requestor ! LoadSnapshotFailure(e, iid) - } - case SaveSnapshot(snapshot, initiator, requestor, iid) => - import context.dispatcher - snapshotStore.saveAsync(snapshot) onComplete { - case Success(_) => requestor.tell(SaveSnapshotSuccess(snapshot.metadata, iid), initiator) - case Failure(e) => requestor.tell(SaveSnapshotFailure(snapshot.metadata, e, iid), initiator) - } - case DeleteSnapshots(lowerSequenceNr) => - import context.dispatcher - val sdr = sender() - snapshotStore.deleteAsync(lowerSequenceNr) onComplete { - case Success(_) => sdr ! DeleteSnapshotsSuccess - case Failure(e) => sdr ! DeleteSnapshotsFailure(e) - } - case Terminated(requestor) => - registry = registry.unregisterSubscriber(requestor) + private var indexSequenceNr: Long = 0L + private var updateCount: Long = 0L + + override def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long): CassandraEventIteratorParameters = + CassandraEventIteratorParameters(fromSequenceNr, indexSequenceNr, toSequenceNr, None) + + override def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long, aggregateId: String): CassandraEventIteratorParameters = + CassandraEventIteratorParameters(fromSequenceNr, indexSequenceNr, toSequenceNr, Some(aggregateId)) + + override def eventIterator(parameters: CassandraEventIteratorParameters): Iterator[DurableEvent] with Closeable = parameters match { + case CassandraEventIteratorParameters(from, idx, to, Some(aggregateId)) => compositeEventIterator(aggregateId, from, idx, to) + case CassandraEventIteratorParameters(from, idx, to, None) => eventLogStore.eventIterator(from, to) } - override def receive = - initializing + override def readReplicationProgress(logId: String): Future[Long] = + progressStore.readReplicationProgressAsync(logId) - private[eventuate] def currentSystemTime: Long = - System.currentTimeMillis + override def readReplicationProgresses: Future[Map[String, Long]] = + progressStore.readReplicationProgressesAsync - private[eventuate] def createEventReader(cassandra: Cassandra, logId: String) = - new CassandraEventReader(cassandra, logId) + override def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: (DurableEvent) => Boolean): Future[ReadResult] = + eventLogStore.readAsync(fromSequenceNr, toSequenceNr, max, filter) - private[eventuate] def createIndex(cassandra: Cassandra, eventReader: CassandraEventReader, logId: String) = - context.actorOf(CassandraIndex.props(cassandra, eventReader, logId)) + override def recoverClockSuccess(clock: Clock): Unit = + indexSequenceNr = clock.sequenceNr - private[eventuate] def createReplicationProgressStore(cassandra: Cassandra, logId: String) = - new CassandraReplicationProgressStore(cassandra, logId) + override def recoverClock: Future[Clock] = { + implicit val timeout = Timeout(Int.MaxValue /* FIXME */ , TimeUnit.MILLISECONDS) + import services.readDispatcher - private[eventuate] def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable, fromSequenceNr: Long): ActorRef = - context.actorOf(Props(new ChunkedEventReplay(requestor, iterator)).withDispatcher("eventuate.log.cassandra.read-dispatcher")) - - private def replicateN(writes: Seq[ReplicationWrite]): Unit = { - writes.foreach(w => timeCache = timeCache.updated(w.sourceLogId, w.currentSourceVectorTime)) - val result = for { - (partition, tracker) <- Try(adjustSequenceNr(writes.map(_.events.size).sum, cassandra.settings.partitionSizeMax, timeTracker)) - (updatedWrites, tracker2) = tracker.prepareReplicates(id, writes, log) - updatedEvents = updatedWrites.flatMap(_.events) - tracker3 <- Try(write(partition, updatedEvents, tracker2)) - } yield (updatedWrites, updatedEvents, tracker3) - - result match { - case Success((updatedWrites, updatedEvents, tracker3)) => - timeTracker = tracker3 - updatedWrites.foreach { w => - val rws = ReplicationWriteSuccess(w.events.size, w.replicationProgress, tracker3.vectorTime) - val sdr = w.initiator - registry.pushReplicateSuccess(w.events) - notificationChannel ! rws - implicit val dispatcher = context.system.dispatchers.defaultGlobalDispatcher - progressStore.writeReplicationProgressAsync(w.sourceLogId, w.replicationProgress) onComplete { - case Success(_) => - sdr ! rws - case Failure(e) => - // Write failure of replication progress can be ignored. Using a stale - // progress to resume replication will redundantly read events from a - // source log but these events will be successfully identified as - // duplicates, either at source or latest at target. - log.warning(s"Writing of replication progress failed: ${e.getMessage}") - sdr ! ReplicationWriteFailure(e) - } - } - notificationChannel ! Updated(updatedEvents) - case Failure(e) => - writes.foreach { w => - w.initiator ! ReplicationWriteFailure(e) - } + index ? InitIndex flatMap { + case InitIndexSuccess(c) => Future.successful(c) + case InitIndexFailure(e) => Future.failed(e) } } - private def writeN(writes: Seq[Write]): Unit = { - val result = for { - (partition, tracker) <- Try(adjustSequenceNr(writes.map(_.events.size).sum, cassandra.settings.partitionSizeMax, timeTracker)) - (updatedWrites, tracker2) = tracker.prepareWrites(id, writes, currentSystemTime) - updatedEvents = updatedWrites.flatMap(_.events) - tracker3 <- Try(write(partition, updatedEvents, tracker2)) - } yield (updatedWrites, updatedEvents, tracker3) - - result match { - case Success((updatedWrites, updatedEvents, tracker3)) => - timeTracker = tracker3 - updatedWrites.foreach(w => registry.pushWriteSuccess(w.events, w.initiator, w.requestor, w.instanceId)) - notificationChannel ! Updated(updatedEvents) - case Failure(e) => - writes.foreach(w => registry.pushWriteFailure(w.events, w.initiator, w.requestor, w.instanceId, e)) + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = { + eventLogStore.writeSync(events, partition) + updateCount += events.size + if (updateCount >= cassandra.settings.indexUpdateLimit) { + index ! UpdateIndex(null, clock.sequenceNr) + updateCount = 0L } } - private[eventuate] def write(partition: Long, events: Seq[DurableEvent], tracker: TimeTracker): TimeTracker = { - cassandra.executeBatch { batch => - events.foreach { event => - batch.add(preparedWriteEventStatement.bind(partition: JLong, event.localSequenceNr: JLong, cassandra.eventToByteBuffer(event))) - } - } - if (tracker.updateCount >= cassandra.settings.indexUpdateLimit) { - index ! Update(tracker.sequenceNr) - tracker.copy(updateCount = 0L) - } else { - tracker - } + override def writeReplicationProgress(logId: String, progress: Long): Future[Unit] = { + implicit val dispatcher = context.system.dispatchers.defaultGlobalDispatcher + progressStore.writeReplicationProgressAsync(logId, progress) } -} -object CassandraEventLog { - import scala.language.postfixOps + override def unhandled(message: Any): Unit = message match { + case u @ UpdateIndexSuccess(clock, _) => + indexSequenceNr = clock.sequenceNr + onIndexEvent(u) + case u @ UpdateIndexFailure(_) => + onIndexEvent(u) + case r @ ReadClockFailure(_) => + onIndexEvent(r) + case other => + super.unhandled(other) + } - private[eventuate] case class Initialize(timeTracker: TimeTracker) + private[eventuate] def createIndex(cassandra: Cassandra, indexStore: CassandraIndexStore, eventLogStore: CassandraEventLogStore, logId: String) = + context.actorOf(CassandraIndex.props(cassandra, eventLogStore, indexStore, logId)) - private lazy val validCassandraIdentifier = "^[a-zA-Z0-9_]+$"r + private[eventuate] def createIndexStore(cassandra: Cassandra, logId: String) = + new CassandraIndexStore(cassandra, logId) - /** - * Adjusts `timeTracker.sequenceNumber` if a batch of `batchSize` doesn't fit in the current partition. - */ - def adjustSequenceNr(batchSize: Long, maxBatchSize: Long, tracker: TimeTracker): (Long, TimeTracker) = { - require(batchSize <= maxBatchSize, s"write batch size (${batchSize}) must not be greater than maximum partition size (${maxBatchSize})") - - val currentPartition = partitionOf(tracker.sequenceNr, maxBatchSize) - val remainingPartitionSize = partitionSize(tracker.sequenceNr, maxBatchSize) - if (remainingPartitionSize < batchSize) { - (currentPartition + 1L, tracker.advanceSequenceNr(remainingPartitionSize)) - } else { - (currentPartition, tracker) + private[eventuate] def createEventLogStore(cassandra: Cassandra, logId: String) = + new CassandraEventLogStore(cassandra, logId) + + private[eventuate] def createReplicationProgressStore(cassandra: Cassandra, logId: String) = + new CassandraReplicationProgressStore(cassandra, logId) + + private def compositeEventIterator(aggregateId: String, fromSequenceNr: Long, indexSequenceNr: Long, toSequenceNr: Long): Iterator[DurableEvent] with Closeable = + new CompositeEventIterator(aggregateId, fromSequenceNr, indexSequenceNr, toSequenceNr) + + private class CompositeEventIterator(aggregateId: String, fromSequenceNr: Long, indexSequenceNr: Long, toSequenceNr: Long) extends Iterator[DurableEvent] with Closeable { + private var iter: Iterator[DurableEvent] = indexStore.aggregateEventIterator(aggregateId, fromSequenceNr, indexSequenceNr) + private var last = fromSequenceNr - 1L + private var idxr = true + + @annotation.tailrec + final def hasNext: Boolean = + if (idxr && iter.hasNext) { + true + } else if (idxr) { + idxr = false + iter = eventLogStore.eventIterator((indexSequenceNr max last) + 1L, toSequenceNr).filter(_.destinationAggregateIds.contains(aggregateId)) + hasNext + } else { + iter.hasNext + } + + def next(): DurableEvent = { + val evt = iter.next() + last = evt.localSequenceNr + evt } + + def close(): Unit = + () } + // ------------------------------------------------------------------ + // Test support + // ------------------------------------------------------------------ + + private[eventuate] def onIndexEvent(event: Any): Unit = () +} + +object CassandraEventLog { + val validCassandraIdentifier = "^[a-zA-Z0-9_]+$"r + /** * Check whether the specified `logId` is valid for Cassandra * table, column and/or keyspace name usage. @@ -320,32 +204,6 @@ object CassandraEventLog { def isValidEventLogId(logId: String): Boolean = validCassandraIdentifier.findFirstIn(logId).isDefined - /** - * Partition number for given `sequenceNr`. - */ - def partitionOf(sequenceNr: Long, partitionSizeMax: Long): Long = - if (sequenceNr == 0L) -1L else (sequenceNr - 1L) / partitionSizeMax - - /** - * Remaining partition size given the current `sequenceNr`. - */ - def partitionSize(sequenceNr: Long, partitionSizeMax: Long): Long = { - val m = sequenceNr % partitionSizeMax - if (m == 0L) m else partitionSizeMax - m - } - - /** - * First sequence number of given `partition`. - */ - def firstSequenceNr(partition: Long, partitionSizeMax: Long): Long = - partition * partitionSizeMax + 1L - - /** - * Last sequence number of given `partition`. - */ - def lastSequenceNr(partition: Long, partitionSizeMax: Long): Long = - (partition + 1L) * partitionSizeMax - /** * Creates a [[CassandraEventLog]] configuration object. * @@ -354,6 +212,6 @@ object CassandraEventLog { */ def props(logId: String, batching: Boolean = true): Props = { val logProps = Props(new CassandraEventLog(logId)).withDispatcher("eventuate.log.cassandra.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + if (batching) Props(new BatchingLayer(logProps)) else logProps } -} +} \ No newline at end of file diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraSettings.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogSettings.scala similarity index 86% rename from src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraSettings.scala rename to src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogSettings.scala index 09c3601c..d002d60f 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraSettings.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogSettings.scala @@ -25,17 +25,19 @@ import com.datastax.driver.core.{ Cluster, ConsistencyLevel } import com.typesafe.config.Config import com.rbmhtechnology.eventuate.ReplicationSettings -import com.rbmhtechnology.eventuate.log.BatchingSettings +import com.rbmhtechnology.eventuate.log._ import scala.collection.JavaConverters._ import scala.concurrent.duration._ -private[eventuate] class CassandraSettings(config: Config) { - import CassandraSettings._ +class CassandraEventLogSettings(config: Config) extends EventLogSettings { + import CassandraEventLogSettings._ private val batchingSettings = new BatchingSettings(config) private val replicationSettings = new ReplicationSettings(config) + val readDispatcherId: String = "eventuate.log.cassandra.read-dispatcher" + val keyspace: String = config.getString("eventuate.log.cassandra.keyspace") @@ -60,19 +62,22 @@ private[eventuate] class CassandraSettings(config: Config) { val contactPoints = getContactPoints(config.getStringList("eventuate.log.cassandra.contact-points").asScala, defaultPort) - val partitionSizeMax: Int = - config.getInt("eventuate.log.cassandra.partition-size-max") + val partitionSizeMax: Long = + config.getLong("eventuate.log.cassandra.partition-size-max") .requiring(_ > batchingSettings.batchSizeLimit, s"eventuate.log.cassandra.partition-size-max must be greater than eventuate.log.batching.batch-size-limit (${batchingSettings.batchSizeLimit})") .requiring(_ > replicationSettings.batchSizeMax, s"eventuate.log.cassandra.partition-size-max must be greater than eventuate.log.replication.batch-size-max (${replicationSettings.batchSizeMax})") - val initRetryBackoff: FiniteDuration = - config.getDuration("eventuate.log.cassandra.init-retry-backoff", TimeUnit.MILLISECONDS).millis - val indexUpdateLimit: Int = config.getInt("eventuate.log.cassandra.index-update-limit") + val initRetryMax: Int = + config.getInt("eventuate.log.cassandra.init-retry-max") + + val initRetryDelay: FiniteDuration = + config.getDuration("eventuate.log.cassandra.init-retry-delay", TimeUnit.MILLISECONDS).millis + val initialConnectRetryMax: Int = config.getInt("eventuate.log.cassandra.initial-connect-retry-max") @@ -85,7 +90,7 @@ private[eventuate] class CassandraSettings(config: Config) { config.getString("eventuate.log.cassandra.password")) } -private object CassandraSettings { +private object CassandraEventLogSettings { def getContactPoints(contactPoints: Seq[String], defaultPort: Int): Seq[InetSocketAddress] = { contactPoints match { case null | Nil => throw new IllegalArgumentException("a contact point list cannot be empty.") diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventReader.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogStore.scala similarity index 80% rename from src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventReader.scala rename to src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogStore.scala index addd24aa..2ff2e619 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventReader.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraEventLogStore.scala @@ -20,31 +20,40 @@ import java.io.Closeable import java.lang.{ Long => JLong } import com.datastax.driver.core.{ PreparedStatement, Row } -import com.rbmhtechnology.eventuate.ReplicationFilter.NoFilter import com.rbmhtechnology.eventuate._ +import com.rbmhtechnology.eventuate.log.EventLog import scala.collection.JavaConverters._ import scala.collection.immutable.Seq import scala.concurrent.Future -private[eventuate] class CassandraEventReader(cassandra: Cassandra, logId: String) { - import CassandraEventReader._ - import CassandraEventLog._ +private[eventuate] class CassandraEventLogStore(cassandra: Cassandra, logId: String) { + import EventLog._ + + val preparedWriteEventStatement: PreparedStatement = + cassandra.prepareWriteEvent(logId) val preparedReadEventsStatement: PreparedStatement = cassandra.prepareReadEvents(logId) + def writeSync(events: Seq[DurableEvent], partition: Long) = + cassandra.executeBatch { batch => + events.foreach { event => + batch.add(preparedWriteEventStatement.bind(partition: JLong, event.localSequenceNr: JLong, cassandra.eventToByteBuffer(event))) + } + } + def readAsync(fromSequenceNr: Long, toSequenceNr: Long, max: Int): Future[ReadResult] = - readAsync(fromSequenceNr, toSequenceNr, max, NoFilter, VectorTime.Zero, logId) + readAsync(fromSequenceNr, toSequenceNr, max, _ => true) - def readAsync(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: ReplicationFilter, lower: VectorTime, targetLogId: String): Future[ReadResult] = - Future(read(fromSequenceNr, toSequenceNr, max, filter, lower, targetLogId))(cassandra.readDispatcher) + def readAsync(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: DurableEvent => Boolean): Future[ReadResult] = + Future(read(fromSequenceNr, toSequenceNr, max, filter))(cassandra.readDispatcher) - def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: ReplicationFilter, lower: VectorTime, targetLogId: String): ReadResult = { + def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: DurableEvent => Boolean): ReadResult = { var lastSequenceNr = fromSequenceNr - 1L val events = eventIterator(fromSequenceNr, toSequenceNr).filter { evt => lastSequenceNr = evt.localSequenceNr - evt.replicable(lower, filter) + filter(evt) }.take(max).toVector ReadResult(events, lastSequenceNr) } @@ -95,7 +104,3 @@ private[eventuate] class CassandraEventReader(cassandra: Cassandra, logId: Strin () } } - -private[eventuate] object CassandraEventReader { - case class ReadResult(events: Seq[DurableEvent], to: Long) -} diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndex.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndex.scala index b75271ff..d1bff2d3 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndex.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndex.scala @@ -16,150 +16,91 @@ package com.rbmhtechnology.eventuate.log.cassandra -import java.io.Closeable -import java.lang.{ Long => JLong } - import akka.actor._ -import com.datastax.driver.core._ import com.rbmhtechnology.eventuate._ -import com.rbmhtechnology.eventuate.EventsourcingProtocol._ -import com.rbmhtechnology.eventuate.log._ +import com.rbmhtechnology.eventuate.log.EventLog.Clock -import scala.collection.JavaConverters._ import scala.collection.immutable.Seq import scala.concurrent._ import scala.util._ -private[eventuate] class CassandraIndex(cassandra: Cassandra, eventReader: CassandraEventReader, logId: String) extends Actor with Stash with ActorLogging { +private[eventuate] class CassandraIndex(cassandra: Cassandra, eventLogStore: CassandraEventLogStore, indexStore: CassandraIndexStore, logId: String) extends Actor with Stash with ActorLogging { import CassandraIndex._ - import CassandraEventLog._ import context.dispatcher private val scheduler = context.system.scheduler private val eventLog = context.parent - private val indexStore = createIndexStore(cassandra, logId) - private val indexUpdater = context.actorOf(Props(new CassandraIndexUpdater(cassandra, eventReader, indexStore))) + private val indexUpdater = context.actorOf(Props(new CassandraIndexUpdater(cassandra, eventLogStore, indexStore))) /** * Contains the sequence number of the last event in event log that * has been successfully processed and written to the index. */ - private var timeTracker: TimeTracker = TimeTracker() + private var clock: Clock = Clock() + + def instantiated: Receive = { + case InitIndex => + indexStore.readClockAsync onComplete { + case Success(t) => self ! ReadClockSuccess(t) + case Failure(e) => self ! ReadClockFailure(e) + } + context.become(initializing(sender())) + } - def initializing: Receive = { - case ReadTimeTracker => - indexStore.readTimeTrackerAsync onComplete { - case Success(t) => self ! ReadTimeTrackerSuccess(t) - case Failure(e) => self ! ReadTimeTrackerFailure(e) + def initializing(sdr: ActorRef): Receive = { + case InitIndex => + indexStore.readClockAsync onComplete { + case Success(t) => self ! ReadClockSuccess(t) + case Failure(e) => self ! ReadClockFailure(e) } - case ReadTimeTrackerSuccess(t) => + case ReadClockSuccess(t) => indexUpdater ! UpdateIndex(t, Long.MaxValue) case u @ UpdateIndexSuccess(t, _) => - timeTracker = t - eventLog ! Initialize(t) + clock = t + eventLog ! u + sdr ! InitIndexSuccess(t) context.become(initialized) - unstashAll() - onIndexEvent(u) case u @ UpdateIndexFailure(cause) => - log.error(cause, "UpdateIndex failure. Retry ...") - scheduleReadTimeTracker() - onIndexEvent(u) - case r @ ReadTimeTrackerFailure(cause) => - log.error(cause, "ReadTimeTracker failed. Retry ...") - scheduleReadTimeTracker() - onIndexEvent(r) - case other => - stash() + log.error(cause, "UpdateIndex failure") + eventLog ! u + sdr ! InitIndexFailure(cause) + context.become(instantiated) + case r @ ReadClockFailure(cause) => + log.error(cause, "ReadClock failure") + eventLog ! r + sdr ! InitIndexFailure(cause) + context.become(instantiated) } def initialized: Receive = { - case Update(toSequenceNr) => - indexUpdater ! UpdateIndex(timeTracker, toSequenceNr) + case UpdateIndex(_, toSequenceNr) => + indexUpdater ! UpdateIndex(clock, toSequenceNr) case u @ UpdateIndexSuccess(t, _) => - timeTracker = t - onIndexEvent(u) + clock = t + eventLog ! u case u @ UpdateIndexFailure(cause) => log.error(cause, "UpdateIndex failure") - onIndexEvent(u) - case ReplayIndex(fromSequenceNr, toSequenceNr, max, requestor, emitterAggregateId, iid) => - val indexSequenceNr = timeTracker.sequenceNr // avoid async evaluation by replayer actor - replayer(requestor, compositeEventIterator(emitterAggregateId, fromSequenceNr, indexSequenceNr, toSequenceNr)) ! ReplayNext(max, iid) + eventLog ! u } def receive = - initializing - - private[eventuate] def createIndexStore(cassandra: Cassandra, logId: String) = - new CassandraIndexStore(cassandra, logId) - - private def scheduleReadTimeTracker(): Unit = - scheduler.scheduleOnce(cassandra.settings.initRetryBackoff, self, ReadTimeTracker) - - private def updateIncrementAsync(increment: IndexIncrement): Future[IndexIncrement] = - Future(updateIncrement(increment))(cassandra.readDispatcher) - - private def updateIncrement(increment: IndexIncrement): IndexIncrement = { - eventReader.eventIterator(increment.timeTracker.sequenceNr + 1L, Long.MaxValue).foldLeft(increment) { - case (inc, event) => inc.update(event) - } - } - - private def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable): ActorRef = - context.actorOf(Props(new ChunkedEventReplay(requestor, iterator)).withDispatcher("eventuate.log.cassandra.read-dispatcher")) - - private def compositeEventIterator(aggregateId: String, fromSequenceNr: Long, indexSequenceNr: Long, toSequenceNr: Long): Iterator[DurableEvent] with Closeable = - new CompositeEventIterator(aggregateId, fromSequenceNr, indexSequenceNr, toSequenceNr) - - private class CompositeEventIterator(aggregateId: String, fromSequenceNr: Long, indexSequenceNr: Long, toSequenceNr: Long) extends Iterator[DurableEvent] with Closeable { - private var iter: Iterator[DurableEvent] = indexStore.aggregateEventIterator(aggregateId, fromSequenceNr, indexSequenceNr) - private var last = fromSequenceNr - 1L - private var idxr = true - - @annotation.tailrec - final def hasNext: Boolean = - if (idxr && iter.hasNext) { - true - } else if (idxr) { - idxr = false - iter = eventReader.eventIterator((indexSequenceNr max last) + 1L, toSequenceNr).filter(_.destinationAggregateIds.contains(aggregateId)) - hasNext - } else { - iter.hasNext - } - - def next(): DurableEvent = { - val evt = iter.next() - last = evt.localSequenceNr - evt - } - - def close(): Unit = - () - } - - override def preStart(): Unit = - self ! ReadTimeTracker - - // ------------------------------------------------------------------ - // Test support - // ------------------------------------------------------------------ - - def onIndexEvent(event: Any): Unit = () + instantiated } private[eventuate] object CassandraIndex { - case class ReplayIndex(from: Long, to: Long, max: Int, requestor: ActorRef, aggregateId: String, instanceId: Int) + case object InitIndex + case class InitIndexSuccess(clock: Clock) + case class InitIndexFailure(cause: Throwable) - case object ReadTimeTracker - case class ReadTimeTrackerSuccess(timeTracker: TimeTracker) - case class ReadTimeTrackerFailure(cause: Throwable) + case object ReadClock + case class ReadClockSuccess(clock: Clock) + case class ReadClockFailure(cause: Throwable) - case class Update(toSequenceNr: Long) - case class UpdateIndex(timeTracker: TimeTracker, toSequenceNr: Long) + case class UpdateIndex(clock: Clock, toSequenceNr: Long) case class UpdateIndexProgress(increment: IndexIncrement) - case class UpdateIndexSuccess(timeTracker: TimeTracker, steps: Int = 0) + case class UpdateIndexSuccess(clock: Clock, steps: Int = 0) case class UpdateIndexFailure(cause: Throwable) case class AggregateEvents(events: Map[String, Vector[DurableEvent]] = Map.empty) { @@ -172,31 +113,31 @@ private[eventuate] object CassandraIndex { }) } - case class IndexIncrement(aggregateEvents: AggregateEvents, timeTracker: TimeTracker) { + case class IndexIncrement(aggregateEvents: AggregateEvents, clock: Clock) { def update(events: Seq[DurableEvent]): IndexIncrement = events.foldLeft(this) { case (acc, event) => acc.update(event) } def update(event: DurableEvent): IndexIncrement = - copy(aggregateEvents.update(event), timeTracker.update(event)) + copy(aggregateEvents.update(event), clock.update(event)) def clearAggregateEvents: IndexIncrement = { - copy(AggregateEvents(), timeTracker) + copy(AggregateEvents(), clock) } } - def props(cassandra: Cassandra, eventReader: CassandraEventReader, logId: String): Props = - Props(new CassandraIndex(cassandra, eventReader, logId: String)) + def props(cassandra: Cassandra, eventLogStore: CassandraEventLogStore, indexStore: CassandraIndexStore, logId: String): Props = + Props(new CassandraIndex(cassandra, eventLogStore, indexStore, logId)) } -private[eventuate] class CassandraIndexUpdater(cassandra: Cassandra, eventReader: CassandraEventReader, indexStore: CassandraIndexStore) extends Actor { +private class CassandraIndexUpdater(cassandra: Cassandra, eventLogStore: CassandraEventLogStore, indexStore: CassandraIndexStore) extends Actor { import CassandraIndex._ import context.dispatcher val index = context.parent val idle: Receive = { - case UpdateIndex(timeTracker, toSequenceNr) => - update(timeTracker.sequenceNr + 1L, toSequenceNr, IndexIncrement(AggregateEvents(), timeTracker)) + case UpdateIndex(clock, toSequenceNr) => + update(clock.sequenceNr + 1L, toSequenceNr, IndexIncrement(AggregateEvents(), clock)) context.become(updating(0, toSequenceNr)) } @@ -208,7 +149,7 @@ private[eventuate] class CassandraIndexUpdater(cassandra: Cassandra, eventReader index ! UpdateIndexSuccess(t, steps) context.become(idle) case UpdateIndexProgress(inc) => - update(inc.timeTracker.sequenceNr + 1L, toSequenceNr, inc.clearAggregateEvents) + update(inc.clock.sequenceNr + 1L, toSequenceNr, inc.clearAggregateEvents) context.become(updating(steps + 1, toSequenceNr)) } @@ -217,84 +158,16 @@ private[eventuate] class CassandraIndexUpdater(cassandra: Cassandra, eventReader def update(fromSequenceNr: Long, toSequenceNr: Long, increment: IndexIncrement): Unit = updateAsync(fromSequenceNr, toSequenceNr, increment) onComplete { case Success((inc, true)) => self ! UpdateIndexProgress(inc) - case Success((inc, false)) => self ! UpdateIndexSuccess(inc.timeTracker) + case Success((inc, false)) => self ! UpdateIndexSuccess(inc.clock) case Failure(err) => self ! UpdateIndexFailure(err) } def updateAsync(fromSequenceNr: Long, toSequenceNr: Long, increment: IndexIncrement): Future[(IndexIncrement, Boolean)] = for { - res <- eventReader.readAsync(fromSequenceNr, toSequenceNr, cassandra.settings.indexUpdateLimit) + res <- eventLogStore.readAsync(fromSequenceNr, toSequenceNr, cassandra.settings.indexUpdateLimit) inc <- writeAsync(increment.update(res.events)) } yield (inc, res.events.nonEmpty) def writeAsync(increment: IndexIncrement): Future[IndexIncrement] = - indexStore.writeAsync(increment.aggregateEvents, increment.timeTracker).map(_ => increment) + indexStore.writeAsync(increment.aggregateEvents, increment.clock).map(_ => increment) } - -private[eventuate] class CassandraIndexStore(cassandra: Cassandra, logId: String) { - import CassandraIndex._ - - private val preparedReadAggregateEventStatement: PreparedStatement = cassandra.prepareReadAggregateEvents(logId) - private val preparedWriteAggregateEventStatement: PreparedStatement = cassandra.prepareWriteAggregateEvent(logId) - - private[eventuate] def readTimeTrackerAsync: Future[TimeTracker] = { - import cassandra.readDispatcher - cassandra.session.executeAsync(cassandra.preparedReadTimeTrackerStatement.bind(logId)).map { resultSet => - if (resultSet.isExhausted) TimeTracker() else cassandra.timeTrackerFromByteBuffer(resultSet.one().getBytes("time_tracker")) - } - } - - private[eventuate] def writeAsync(aggregateEvents: AggregateEvents, timeTracker: TimeTracker)(implicit executor: ExecutionContext): Future[TimeTracker] = { - for { - _ <- writeAggregateEventsAsync(aggregateEvents) - t <- writeTimeTrackerAsync(timeTracker) // must be after other writes - } yield t - } - - private def writeAggregateEventsAsync(aggregateEvents: AggregateEvents)(implicit executor: ExecutionContext): Future[Unit] = - Future.sequence(aggregateEvents.events.map { - case (aggregateId, events) => writeAggregateEventsAsync(aggregateId, events) - }).map(_ => ()) - - private def writeAggregateEventsAsync(aggregateId: String, events: Seq[DurableEvent])(implicit executor: ExecutionContext): Future[Unit] = cassandra.executeBatchAsync { batch => - events.foreach(event => batch.add(preparedWriteAggregateEventStatement.bind(aggregateId, event.localSequenceNr: JLong, cassandra.eventToByteBuffer(event)))) - } - - private def writeTimeTrackerAsync(timeTracker: TimeTracker)(implicit executor: ExecutionContext): Future[TimeTracker] = - cassandra.session.executeAsync(cassandra.preparedWriteTimeTrackerStatement.bind(logId, cassandra.timeTrackerToByteBuffer(timeTracker))).map(_ => timeTracker) - - private[eventuate] def aggregateEventIterator(aggregateId: String, fromSequenceNr: Long, toSequenceNr: Long): Iterator[DurableEvent] = - new AggregateEventIterator(aggregateId, fromSequenceNr, toSequenceNr) - - private class AggregateEventIterator(aggregateId: String, fromSequenceNr: Long, toSequenceNr: Long) extends Iterator[DurableEvent] { - var currentSequenceNr = fromSequenceNr - var currentIter = newIter() - var rowCount = 0 - - def newIter(): Iterator[Row] = - if (currentSequenceNr > toSequenceNr) Iterator.empty else cassandra.session.execute(preparedReadAggregateEventStatement.bind(aggregateId, currentSequenceNr: JLong, toSequenceNr: JLong)).iterator.asScala - - @annotation.tailrec - final def hasNext: Boolean = { - if (currentIter.hasNext) { - true - } else if (rowCount < cassandra.settings.partitionSizeMax) { - // all events consumed - false - } else { - // max result set size reached, fetch again - currentSequenceNr += 1L - currentIter = newIter() - rowCount = 0 - hasNext - } - } - - def next(): DurableEvent = { - val row = currentIter.next() - currentSequenceNr = row.getLong("sequence_nr") - rowCount += 1 - cassandra.eventFromByteBuffer(row.getBytes("event")) - } - } -} \ No newline at end of file diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndexStore.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndexStore.scala new file mode 100644 index 00000000..cea887f0 --- /dev/null +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraIndexStore.scala @@ -0,0 +1,96 @@ +/* + * Copyright (C) 2015 Red Bull Media House GmbH - all rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.rbmhtechnology.eventuate.log.cassandra + +import java.lang.{ Long => JLong } + +import com.datastax.driver.core._ + +import com.rbmhtechnology.eventuate.DurableEvent +import com.rbmhtechnology.eventuate.log.EventLog.Clock + +import scala.collection.JavaConverters._ +import scala.collection.immutable.Seq +import scala.concurrent._ + +private[eventuate] class CassandraIndexStore(cassandra: Cassandra, logId: String) { + import CassandraIndex._ + + private val preparedReadAggregateEventStatement: PreparedStatement = cassandra.prepareReadAggregateEvents(logId) + private val preparedWriteAggregateEventStatement: PreparedStatement = cassandra.prepareWriteAggregateEvent(logId) + + private[eventuate] def readClockAsync: Future[Clock] = { + import cassandra.readDispatcher + cassandra.session.executeAsync(cassandra.preparedReadClockStatement.bind(logId)).map { resultSet => + if (resultSet.isExhausted) Clock() else cassandra.clockFromByteBuffer(resultSet.one().getBytes("clock")) + } + } + + private[eventuate] def writeAsync(aggregateEvents: AggregateEvents, clock: Clock)(implicit executor: ExecutionContext): Future[Clock] = { + for { + _ <- writeAggregateEventsAsync(aggregateEvents) + t <- writeClockAsync(clock) // must be after other writes + } yield t + } + + private def writeAggregateEventsAsync(aggregateEvents: AggregateEvents)(implicit executor: ExecutionContext): Future[Unit] = + Future.sequence(aggregateEvents.events.map { + case (aggregateId, events) => writeAggregateEventsAsync(aggregateId, events) + }).map(_ => ()) + + private def writeAggregateEventsAsync(aggregateId: String, events: Seq[DurableEvent])(implicit executor: ExecutionContext): Future[Unit] = cassandra.executeBatchAsync { batch => + events.foreach(event => batch.add(preparedWriteAggregateEventStatement.bind(aggregateId, event.localSequenceNr: JLong, cassandra.eventToByteBuffer(event)))) + } + + private def writeClockAsync(clock: Clock)(implicit executor: ExecutionContext): Future[Clock] = + cassandra.session.executeAsync(cassandra.preparedWriteClockStatement.bind(logId, cassandra.clockToByteBuffer(clock))).map(_ => clock) + + private[eventuate] def aggregateEventIterator(aggregateId: String, fromSequenceNr: Long, toSequenceNr: Long): Iterator[DurableEvent] = + new AggregateEventIterator(aggregateId, fromSequenceNr, toSequenceNr) + + private class AggregateEventIterator(aggregateId: String, fromSequenceNr: Long, toSequenceNr: Long) extends Iterator[DurableEvent] { + var currentSequenceNr = fromSequenceNr + var currentIter = newIter() + var rowCount = 0 + + def newIter(): Iterator[Row] = + if (currentSequenceNr > toSequenceNr) Iterator.empty else cassandra.session.execute(preparedReadAggregateEventStatement.bind(aggregateId, currentSequenceNr: JLong, toSequenceNr: JLong)).iterator.asScala + + @annotation.tailrec + final def hasNext: Boolean = { + if (currentIter.hasNext) { + true + } else if (rowCount < cassandra.settings.partitionSizeMax) { + // all events consumed + false + } else { + // max result set size reached, fetch again + currentSequenceNr += 1L + currentIter = newIter() + rowCount = 0 + hasNext + } + } + + def next(): DurableEvent = { + val row = currentIter.next() + currentSequenceNr = row.getLong("sequence_nr") + rowCount += 1 + cassandra.eventFromByteBuffer(row.getBytes("event")) + } + } +} \ No newline at end of file diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraStatements.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraStatements.scala index 7e076b61..ccd0d8bd 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraStatements.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/cassandra/CassandraStatements.scala @@ -17,7 +17,7 @@ package com.rbmhtechnology.eventuate.log.cassandra private[eventuate] trait CassandraStatements { - def settings: CassandraSettings + def settings: CassandraEventLogSettings def createKeySpaceStatement = s""" CREATE KEYSPACE IF NOT EXISTS ${settings.keyspace} @@ -80,25 +80,25 @@ private[eventuate] trait CassandraAggregateEventStatements extends CassandraStat def aggregateEventTable(logId: String) = s"${table(logId)}_agg" } -private[eventuate] trait CassandraTimeTrackerStatements extends CassandraStatements { - def createTimeTrackerTableStatement = s""" - CREATE TABLE IF NOT EXISTS ${timeTrackerTable} ( +private[eventuate] trait CassandraClockStatements extends CassandraStatements { + def createClockTableStatement = s""" + CREATE TABLE IF NOT EXISTS ${clockTable} ( log_id text, - time_tracker blob, + clock blob, PRIMARY KEY (log_id)) """ - def writeTimeTrackerStatement = s""" - INSERT INTO ${timeTrackerTable} (log_id, time_tracker) + def writeClockStatement = s""" + INSERT INTO ${clockTable} (log_id, clock) VALUES (?, ?) """ - def readTimeTrackerStatement = s""" - SELECT * FROM ${timeTrackerTable} WHERE + def readClockStatement = s""" + SELECT * FROM ${clockTable} WHERE log_id = ? """ - def timeTrackerTable = table("tt") + def clockTable = table("tt") } private[eventuate] trait CassandraReplicationProgressStatements extends CassandraStatements { diff --git a/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbEventLog.scala b/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbEventLog.scala index 5736237e..91d46017 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbEventLog.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/log/leveldb/LeveldbEventLog.scala @@ -16,25 +16,50 @@ package com.rbmhtechnology.eventuate.log.leveldb -import java.io.{ Closeable, File } +import java.io._ import java.nio.ByteBuffer +import akka.actor._ +import akka.serialization.SerializationExtension + +import com.rbmhtechnology.eventuate.DurableEvent +import com.rbmhtechnology.eventuate.log._ +import com.rbmhtechnology.eventuate.log.EventLog._ +import com.typesafe.config.Config + +import org.fusesource.leveldbjni.JniDBFactory._ +import org.iq80.leveldb._ + import scala.collection.JavaConverters._ import scala.collection.immutable.Seq import scala.concurrent.Future +import scala.concurrent.duration._ import scala.util._ -import akka.actor._ -import akka.serialization.SerializationExtension +class LeveldbEventLogSettings(config: Config) extends EventLogSettings { + val readDispatcherId: String = + "eventuate.log.leveldb.read-dispatcher" -import org.iq80.leveldb._ -import org.fusesource.leveldbjni.JniDBFactory.factory + val rootDir: String = + config.getString("eventuate.log.leveldb.dir") -import com.rbmhtechnology.eventuate._ -import com.rbmhtechnology.eventuate.EventsourcingProtocol._ -import com.rbmhtechnology.eventuate.ReplicationProtocol._ -import com.rbmhtechnology.eventuate.log._ -import com.rbmhtechnology.eventuate.snapshot.filesystem._ + val fsync: Boolean = + config.getBoolean("eventuate.log.leveldb.fsync") + + val stateSnapshotLimit: Int = + config.getInt("eventuate.log.leveldb.state-snapshot-limit") + + val initRetryDelay: FiniteDuration = + Duration.Zero + + val initRetryMax: Int = + 0 + + val partitionSizeMax: Long = + Long.MaxValue +} + +case class LeveldbEventIteratorParameters(fromSequenceNr: Long, classifier: Int) /** * An event log actor with LevelDB as storage backend. The directory containing the LevelDB files @@ -46,176 +71,81 @@ import com.rbmhtechnology.eventuate.snapshot.filesystem._ * @param id unique log id. * @param prefix prefix of the directory that contains the LevelDB files */ -class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLogging { +class LeveldbEventLog(id: String, prefix: String) extends EventLog[LeveldbEventIteratorParameters](id) { import LeveldbEventLog._ - import NotificationChannel._ - import TimeTracker._ - private val eventStream = context.system.eventStream + override val settings = new LeveldbEventLogSettings(context.system.settings.config) private val serialization = SerializationExtension(context.system) - private val leveldbSettings = new LeveldbSettings(context.system) private val leveldbOptions = new Options().createIfMissing(true) - private val leveldbWriteOptions = new WriteOptions().sync(leveldbSettings.fsync).snapshot(false) + private val leveldbWriteOptions = new WriteOptions().sync(settings.fsync).snapshot(false) private def leveldbReadOptions = new ReadOptions().verifyChecksums(false) - private val leveldbDir = new File(leveldbSettings.rootDir, s"${prefix}-${id}"); leveldbDir.mkdirs() + private val leveldbDir = new File(settings.rootDir, s"${prefix}-${id}"); leveldbDir.mkdirs() private val leveldb = factory.open(leveldbDir, leveldbOptions) - private var registry = SubscriberRegistry() - private var timeTracker = TimeTracker() - private var timeCache = Map.empty[String, VectorTime].withDefaultValue(VectorTime.Zero) - - private val notificationChannel = context.actorOf(Props(new NotificationChannel(id))) - private val snapshotStore = new FilesystemSnapshotStore(new FilesystemSnapshotStoreSettings(context.system), id) private val replicationProgressMap = new LeveldbReplicationProgressStore(leveldb, -3, eventLogIdMap.numericId, eventLogIdMap.findId) private val aggregateIdMap = new LeveldbNumericIdentifierStore(leveldb, -1) private val eventLogIdMap = new LeveldbNumericIdentifierStore(leveldb, -2) - // ------------------------------------------------------ - // TODO: consider exchanging only vector time deltas - // - // Messages: - // - // - ReplicationRead - // - ReplicationReadSuccess - // - ReplicationWrite - // - ReplicationWriteSuccess - // - // This optimization might be necessary if many event- - // sourced actors use their own entry in vector clocks. - // ------------------------------------------------------ - - final def receive = { - case GetTimeTracker => - sender() ! GetTimeTrackerSuccess(timeTracker) - case GetReplicationProgresses => - Try(withIterator(iter => replicationProgressMap.readReplicationProgresses(iter))) match { - case Success(r) => sender() ! GetReplicationProgressesSuccess(r) - case Failure(e) => sender() ! GetReplicationProgressesFailure(e) - } - case GetReplicationProgress(sourceLogId) => - Try(replicationProgressMap.readReplicationProgress(sourceLogId)) match { - case Success(r) => sender() ! GetReplicationProgressSuccess(sourceLogId, r, timeTracker.vectorTime) - case Failure(e) => sender() ! GetReplicationProgressFailure(e) - } - case SetReplicationProgress(sourceLogId, progress) => - Try(withBatch(batch => replicationProgressMap.writeReplicationProgress(sourceLogId, progress, batch))) match { - case Success(_) => sender() ! SetReplicationProgressSuccess(sourceLogId, progress) - case Failure(e) => sender() ! SetReplicationProgressFailure(e) - } - case Replay(from, max, requestor, None, iid) => - registry = registry.registerDefaultSubscriber(context.watch(requestor)) - replayer(requestor, eventIterator(1L max from, EventKey.DefaultClassifier), from) ! ReplayNext(max, iid) - case Replay(from, max, requestor, Some(emitterAggregateId), iid) => - val nid = aggregateIdMap.numericId(emitterAggregateId) - registry = registry.registerAggregateSubscriber(context.watch(requestor), emitterAggregateId) - replayer(requestor, eventIterator(1L max from, nid), from) ! ReplayNext(max, iid) - case r @ ReplicationRead(from, max, filter, targetLogId, _, currentTargetVectorTime) => - import leveldbSettings.readDispatcher - val sdr = sender() - notificationChannel ! r - Future(read(from, max, filter, currentTargetVectorTime)) onComplete { - case Success(result) => - val reply = ReplicationReadSuccess(result.events, result.to, targetLogId, null) - self.tell(reply, sdr) - case Failure(cause) => - val reply = ReplicationReadFailure(cause.getMessage, targetLogId) - sdr ! reply - notificationChannel ! reply - } - case r @ ReplicationReadSuccess(events, _, targetLogId, _) => - // Post-filter events using a possibly updated vector time received from the target. - // This is an optimization to reduce network bandwidth usage. If omitted, events are - // still filtered at target based on the current local vector time at the target (for - // correctness). - val currentTargetVectorTime = timeCache(targetLogId) - val updated = events.filterNot(_.before(currentTargetVectorTime)) - val reply = r.copy(updated, currentSourceVectorTime = timeTracker.vectorTime) - sender() ! reply - notificationChannel ! reply - logFilterStatistics(id, "source", events, updated, log) - case w: Write => - writeN(Seq(w)) - case WriteN(writes) => - writeN(writes) - sender() ! WriteNComplete - case w: ReplicationWrite => - replicateN(Seq(w.copy(initiator = sender()))) - case ReplicationWriteN(writes) => - replicateN(writes) - sender() ! ReplicationWriteNComplete - case LoadSnapshot(emitterId, requestor, iid) => - import leveldbSettings.readDispatcher - snapshotStore.loadAsync(emitterId) onComplete { - case Success(s) => requestor ! LoadSnapshotSuccess(s, iid) - case Failure(e) => requestor ! LoadSnapshotFailure(e, iid) - } - case SaveSnapshot(snapshot, initiator, requestor, iid) => - import context.dispatcher - snapshotStore.saveAsync(snapshot) onComplete { - case Success(_) => requestor.tell(SaveSnapshotSuccess(snapshot.metadata, iid), initiator) - case Failure(e) => requestor.tell(SaveSnapshotFailure(snapshot.metadata, e, iid), initiator) - } - case DeleteSnapshots(lowerSequenceNr) => - import context.dispatcher - val sdr = sender() - snapshotStore.deleteAsync(lowerSequenceNr) onComplete { - case Success(_) => sdr ! DeleteSnapshotsSuccess - case Failure(e) => sdr ! DeleteSnapshotsFailure(e) - } - case Terminated(requestor) => - registry = registry.unregisterSubscriber(requestor) - } + private var updateCount: Long = 0L def logDir: File = leveldbDir - def currentSystemTime: Long = - System.currentTimeMillis - - private def replicateN(writes: Seq[ReplicationWrite]): Unit = { - writes.foreach(w => timeCache = timeCache.updated(w.sourceLogId, w.currentSourceVectorTime)) - val (updatedWrites, tracker) = timeTracker.prepareReplicates(id, writes, log) - val updatedEvents = updatedWrites.flatMap(_.events) - Try { - withBatch { batch => - updatedWrites.foreach(w => replicationProgressMap.writeReplicationProgress(w.sourceLogId, w.replicationProgress, batch)) - write(updatedEvents, tracker, batch) + override def write(events: Seq[DurableEvent], partition: Long, clock: Clock): Unit = + withBatch(batch => writeSync(events, clock, batch)) + + override def writeReplicationProgress(logId: String, progress: Long): Future[Unit] = + completed(withBatch(batch => replicationProgressMap.writeReplicationProgress(logId, progress, batch))) + + override def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long): LeveldbEventIteratorParameters = + LeveldbEventIteratorParameters(fromSequenceNr, EventKey.DefaultClassifier) + + override def eventIteratorParameters(fromSequenceNr: Long, toSequenceNr: Long, aggregateId: String): LeveldbEventIteratorParameters = + LeveldbEventIteratorParameters(fromSequenceNr, aggregateIdMap.numericId(aggregateId)) + + override def eventIterator(settings: LeveldbEventIteratorParameters): Iterator[DurableEvent] with Closeable = + eventIterator(1L max settings.fromSequenceNr, settings.classifier) + + private def eventIterator(from: Long, classifier: Int): EventIterator = + new EventIterator(from, classifier) + + override def readReplicationProgresses: Future[Map[String, Long]] = + completed(withIterator(iter => replicationProgressMap.readReplicationProgresses(iter))) + + override def readReplicationProgress(logId: String): Future[Long] = + completed(withIterator(iter => replicationProgressMap.readReplicationProgress(logId))) + + override def read(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: DurableEvent => Boolean): Future[ReadResult] = + Future(readSync(fromSequenceNr, toSequenceNr, max, filter))(services.readDispatcher) + + override def recoverClock: Future[Clock] = completed { + val snap = leveldb.get(clockKeyBytes) match { + case null => Clock() + case cval => clock(cval) + } + + withEventIterator(snap.sequenceNr + 1L, EventKey.DefaultClassifier) { iter => + iter.foldLeft(snap) { + case (clock, event) => clock.update(event) } - } match { - case Success(tracker2) => - timeTracker = tracker2 - updatedWrites.foreach { w => - registry.pushReplicateSuccess(w.events) - w.initiator ! ReplicationWriteSuccess(w.events.size, w.replicationProgress, tracker2.vectorTime) - notificationChannel ! w - } - notificationChannel ! Updated(updatedEvents) - case Failure(e) => - updatedWrites.foreach { w => - w.initiator ! ReplicationWriteFailure(e) - } } } - private def writeN(writes: Seq[Write]): Unit = { - val (updatedWrites, tracker) = timeTracker.prepareWrites(id, writes, currentSystemTime) - val updatedEvents = updatedWrites.flatMap(_.events) - Try(withBatch(batch => write(updatedEvents, tracker, batch))) match { - case Success(tracker2) => - timeTracker = tracker2 - updatedWrites.foreach(w => registry.pushWriteSuccess(w.events, w.initiator, w.requestor, w.instanceId)) - notificationChannel ! Updated(updatedEvents) - case Failure(e) => - writes.foreach(w => registry.pushWriteFailure(w.events, w.initiator, w.requestor, w.instanceId, e)) + private def readSync(fromSequenceNr: Long, toSequenceNr: Long, max: Int, filter: DurableEvent => Boolean): ReadResult = { + val first = 1L max fromSequenceNr + withEventIterator(first, EventKey.DefaultClassifier) { iter => + var last = first - 1L + val evts = iter.filter { evt => + last = evt.localSequenceNr + filter(evt) + }.take(max).toVector + ReadResult(evts, last) } } - private[eventuate] def write(events: Seq[DurableEvent], tracker: TimeTracker): TimeTracker = - withBatch(write(events, tracker, _)) - - private[eventuate] def write(events: Seq[DurableEvent], tracker: TimeTracker, batch: WriteBatch): TimeTracker = { + private def writeSync(events: Seq[DurableEvent], clock: Clock, batch: WriteBatch): Unit = { events.foreach { event => val sequenceNr = event.localSequenceNr val eventBytes = this.eventBytes(event) @@ -224,29 +154,15 @@ class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLo batch.put(eventKeyBytes(aggregateIdMap.numericId(id), sequenceNr), eventBytes) } } - if (tracker.updateCount >= leveldbSettings.stateSnapshotLimit) { - batch.put(timeTrackerKeyBytes, timeTrackerBytes(tracker)) - tracker.copy(updateCount = 0L) - } else { - tracker - } - } - private[eventuate] def read(from: Long, max: Int, filter: ReplicationFilter, lower: VectorTime): ReadResult = { - val first = 1L max from - withEventIterator(first, EventKey.DefaultClassifier) { iter => - var last = first - 1L - val evts = iter.filter { evt => - last = evt.localSequenceNr - evt.replicable(lower, filter) - }.take(max).toVector - ReadResult(evts, last) + updateCount += events.size + + if (updateCount >= settings.stateSnapshotLimit) { + batch.put(clockKeyBytes, clockBytes(clock)) + updateCount = 0 } } - private[eventuate] def replayer(requestor: ActorRef, iterator: => Iterator[DurableEvent] with Closeable, from: Long): ActorRef = - context.actorOf(Props(new ChunkedEventReplay(requestor, iterator)).withDispatcher("eventuate.log.leveldb.read-dispatcher")) - private def withBatch[R](body: WriteBatch => R): R = { val batch = leveldb.createWriteBatch() try { @@ -280,9 +196,6 @@ class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLo } } - private def eventIterator(from: Long, classifier: Int): EventIterator = - new EventIterator(from, classifier) - private class EventIterator(from: Long, classifier: Int) extends Iterator[DurableEvent] with Closeable { val opts = snapshotOptions() @@ -311,11 +224,11 @@ class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLo private def event(a: Array[Byte]): DurableEvent = serialization.deserialize(a, classOf[DurableEvent]).get - private def timeTrackerBytes(t: TimeTracker): Array[Byte] = - serialization.serialize(t).get + private def clockBytes(clock: Clock): Array[Byte] = + serialization.serialize(clock).get - private def timeTracker(a: Array[Byte]): TimeTracker = - serialization.deserialize(a, classOf[TimeTracker]).get + private def clock(a: Array[Byte]): Clock = + serialization.deserialize(a, classOf[Clock]).get private def snapshotOptions(): ReadOptions = leveldbReadOptions.snapshot(leveldb.getSnapshot) @@ -324,16 +237,7 @@ class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLo withIterator(iter => aggregateIdMap.readIdMap(iter)) withIterator(iter => eventLogIdMap.readIdMap(iter)) leveldb.put(eventKeyEndBytes, Array.empty[Byte]) - leveldb.get(timeTrackerKeyBytes) match { - case null => // use default tracker value - case cval => timeTracker = timeTracker(cval) - } - - withEventIterator(timeTracker.sequenceNr + 1L, EventKey.DefaultClassifier) { iter => - timeTracker = iter.foldLeft(timeTracker) { - case (tracker, event) => tracker.update(event) - } - } + super.preStart() } override def postStop(): Unit = { @@ -376,50 +280,51 @@ class LeveldbEventLog(val id: String, prefix: String) extends Actor with ActorLo } object LeveldbEventLog { - private[eventuate] case class ReadResult(events: Seq[DurableEvent], to: Long) + private case class EventKey(classifier: Int, sequenceNr: Long) - private[eventuate] case class EventKey(classifier: Int, sequenceNr: Long) - - private[eventuate] object EventKey { + private object EventKey { val DefaultClassifier: Int = 0 } - private[eventuate] val timeTrackerKeyBytes: Array[Byte] = + private val clockKeyBytes: Array[Byte] = eventKeyBytes(0, 0L) - private[eventuate] val eventKeyEnd: EventKey = + private val eventKeyEnd: EventKey = EventKey(Int.MaxValue, Long.MaxValue) - private[eventuate] val eventKeyEndBytes: Array[Byte] = + private val eventKeyEndBytes: Array[Byte] = eventKeyBytes(eventKeyEnd.classifier, eventKeyEnd.sequenceNr) - private[eventuate] def eventKeyBytes(classifier: Int, sequenceNr: Long): Array[Byte] = { + private[leveldb] def longBytes(l: Long): Array[Byte] = + ByteBuffer.allocate(8).putLong(l).array + + private[leveldb] def longFromBytes(a: Array[Byte]): Long = + ByteBuffer.wrap(a).getLong + + private def eventKeyBytes(classifier: Int, sequenceNr: Long): Array[Byte] = { val bb = ByteBuffer.allocate(12) bb.putInt(classifier) bb.putLong(sequenceNr) bb.array } - private[eventuate] def eventKey(a: Array[Byte]): EventKey = { + private def eventKey(a: Array[Byte]): EventKey = { val bb = ByteBuffer.wrap(a) EventKey(bb.getInt, bb.getLong) } - private[eventuate] def longBytes(l: Long): Array[Byte] = - ByteBuffer.allocate(8).putLong(l).array - - private[eventuate] def longFromBytes(a: Array[Byte]): Long = - ByteBuffer.wrap(a).getLong + private def completed[A](body: => A): Future[A] = + Future.fromTry(Try(body)) /** * Creates a [[LeveldbEventLog]] configuration object. * * @param logId unique log id. - * @param prefix prefix of the directory that contains the LevelDB files + * @param prefix prefix of the directory that contains the LevelDB files. * @param batching `true` if write-batching shall be enabled (recommended). */ def props(logId: String, prefix: String = "log", batching: Boolean = true): Props = { val logProps = Props(new LeveldbEventLog(logId, prefix)).withDispatcher("eventuate.log.leveldb.write-dispatcher") - if (batching) Props(new BatchingEventLog(logProps)) else logProps + if (batching) Props(new BatchingLayer(logProps)) else logProps } -} +} \ No newline at end of file diff --git a/src/main/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializer.scala b/src/main/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializer.scala index 87c5d664..5fd5b514 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializer.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/serializer/ReplicationProtocolSerializer.scala @@ -100,7 +100,7 @@ class ReplicationProtocolSerializer(system: ExtendedActorSystem) extends Seriali message.events.foreach(event => builder.addEvents(eventSerializer.durableEventFormatBuilder(event))) builder.setReplicationProgress(message.replicationProgress) builder.setTargetLogId(message.targetLogId) - builder.setCurrentSourceVectorTime(eventSerializer.vectorTimeFormatBuilder(message.currentSourceVectorTime)) + builder.setCurrentSourceVersionVector(eventSerializer.vectorTimeFormatBuilder(message.currentSourceVersionVector)) builder } @@ -111,7 +111,7 @@ class ReplicationProtocolSerializer(system: ExtendedActorSystem) extends Seriali builder.setFilter(filterSerializer.filterTreeFormatBuilder(message.filter)) builder.setTargetLogId(message.targetLogId) builder.setReplicator(Serialization.serializedActorPath(message.replicator)) - builder.setCurrentTargetVectorTime(eventSerializer.vectorTimeFormatBuilder(message.currentTargetVectorTime)) + builder.setCurrentTargetVersionVector(eventSerializer.vectorTimeFormatBuilder(message.currentTargetVersionVector)) builder } @@ -152,7 +152,7 @@ class ReplicationProtocolSerializer(system: ExtendedActorSystem) extends Seriali builder.result(), messageFormat.getReplicationProgress, messageFormat.getTargetLogId, - eventSerializer.vectorTime(messageFormat.getCurrentSourceVectorTime)) + eventSerializer.vectorTime(messageFormat.getCurrentSourceVersionVector)) } private def replicationRead(messageFormat: ReplicationReadFormat): ReplicationRead = @@ -162,7 +162,7 @@ class ReplicationProtocolSerializer(system: ExtendedActorSystem) extends Seriali filterSerializer.filterTree(messageFormat.getFilter), messageFormat.getTargetLogId, system.provider.resolveActorRef(messageFormat.getReplicator), - eventSerializer.vectorTime(messageFormat.getCurrentTargetVectorTime)) + eventSerializer.vectorTime(messageFormat.getCurrentTargetVersionVector)) private def replicationReadEnvelope(messageFormat: ReplicationReadEnvelopeFormat): ReplicationReadEnvelope = ReplicationReadEnvelope( diff --git a/src/main/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializer.scala b/src/main/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializer.scala index c7d15b7c..4ae35f88 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializer.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/serializer/SnapshotSerializer.scala @@ -21,7 +21,7 @@ import akka.serialization.Serializer import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.ConfirmedDelivery.DeliveryAttempt -import com.rbmhtechnology.eventuate.log.TimeTracker +import com.rbmhtechnology.eventuate.log.EventLog.Clock import com.rbmhtechnology.eventuate.serializer.SnapshotFormats._ import scala.collection.JavaConverters._ @@ -32,7 +32,7 @@ class SnapshotSerializer(system: ExtendedActorSystem) extends Serializer { val SnapshotClass = classOf[Snapshot] val ConcurrentVersionsTreeClass = classOf[ConcurrentVersionsTree[_, _]] - val TimeTrackerClass = classOf[TimeTracker] + val ClockClass = classOf[Clock] override def identifier: Int = 22566 override def includeManifest: Boolean = true @@ -42,8 +42,8 @@ class SnapshotSerializer(system: ExtendedActorSystem) extends Serializer { snapshotFormatBuilder(s).build().toByteArray case t: ConcurrentVersionsTree[_, _] => concurrentVersionsTreeFormat(t).build().toByteArray - case t: TimeTracker => - timeTrackerFormatBuilder(t).build().toByteArray + case c: Clock => + clockFormatBuilder(c).build().toByteArray case _ => throw new IllegalArgumentException(s"can't serialize object of type ${o.getClass}") } @@ -55,8 +55,8 @@ class SnapshotSerializer(system: ExtendedActorSystem) extends Serializer { snapshot(SnapshotFormat.parseFrom(bytes)) case ConcurrentVersionsTreeClass => concurrentVersionsTree(ConcurrentVersionsTreeFormat.parseFrom(bytes)) - case TimeTrackerClass => - timeTracker(TimeTrackerFormat.parseFrom(bytes)) + case ClockClass => + clock(ClockFormat.parseFrom(bytes)) case _ => throw new IllegalArgumentException(s"can't deserialize object of type ${clazz}") } @@ -115,10 +115,10 @@ class SnapshotSerializer(system: ExtendedActorSystem) extends Serializer { builder.setCreator(versioned.creator) } - private def timeTrackerFormatBuilder(timeTracker: TimeTracker): TimeTrackerFormat.Builder = { - val builder = TimeTrackerFormat.newBuilder - builder.setSequenceNr(timeTracker.sequenceNr) - builder.setVectorTime(eventSerializer.vectorTimeFormatBuilder(timeTracker.vectorTime)) + private def clockFormatBuilder(clock: Clock): ClockFormat.Builder = { + val builder = ClockFormat.newBuilder + builder.setSequenceNr(clock.sequenceNr) + builder.setVersionVector(eventSerializer.vectorTimeFormatBuilder(clock.versionVector)) builder } @@ -171,9 +171,9 @@ class SnapshotSerializer(system: ExtendedActorSystem) extends Serializer { versionedFormat.getCreator) } - private def timeTracker(timeTrackerFormat: TimeTrackerFormat): TimeTracker = { - TimeTracker( - sequenceNr = timeTrackerFormat.getSequenceNr, - vectorTime = eventSerializer.vectorTime(timeTrackerFormat.getVectorTime)) + private def clock(clockFormat: ClockFormat): Clock = { + Clock( + sequenceNr = clockFormat.getSequenceNr, + versionVector = eventSerializer.vectorTime(clockFormat.getVersionVector)) } } diff --git a/src/main/scala/com/rbmhtechnology/eventuate/snapshot/filesystem/FilesystemSnapshotStore.scala b/src/main/scala/com/rbmhtechnology/eventuate/snapshot/filesystem/FilesystemSnapshotStore.scala index 55377fa1..4e4cfbbb 100644 --- a/src/main/scala/com/rbmhtechnology/eventuate/snapshot/filesystem/FilesystemSnapshotStore.scala +++ b/src/main/scala/com/rbmhtechnology/eventuate/snapshot/filesystem/FilesystemSnapshotStore.scala @@ -19,7 +19,7 @@ package com.rbmhtechnology.eventuate.snapshot.filesystem import java.io._ import java.net.URLEncoder -import akka.event.{LogSource, Logging} +import akka.event.{ LogSource, Logging } import com.rbmhtechnology.eventuate._ import com.rbmhtechnology.eventuate.snapshot.SnapshotStore import org.apache.commons.io.IOUtils @@ -33,12 +33,12 @@ object FilesystemSnapshotStore { } /** - * A snapshot store that saves snapshots to the local filesystem. It keeps a configurable - * maximum number of snapshots per event-sourced actor, view, writer or processor. Snapshot - * loading falls back to older snapshots if newer snapshots cannot be loaded. - * - * @see Configuration key `eventuate.snapshot.filesystem.snapshots-per-emitter-max`. - */ + * A snapshot store that saves snapshots to the local filesystem. It keeps a configurable + * maximum number of snapshots per event-sourced actor, view, writer or processor. Snapshot + * loading falls back to older snapshots if newer snapshots cannot be loaded. + * + * @see Configuration key `eventuate.snapshot.filesystem.snapshots-per-emitter-max`. + */ class FilesystemSnapshotStore(settings: FilesystemSnapshotStoreSettings, logId: String) extends SnapshotStore { private val log = Logging(settings.system, classOf[FilesystemSnapshotStore]) private val rootDir = new File(settings.rootDir, URLEncoder.encode(logId, "UTF-8")) @@ -61,8 +61,8 @@ class FilesystemSnapshotStore(settings: FilesystemSnapshotStoreSettings, logId: } def delete(lowerSequenceNr: Long): Unit = for { - emitterId <- rootDir.listFiles - emitterDir = dstDir(emitterId.getName) + emitterId <- rootDir.listFiles + emitterDir = dstDir(emitterId.getName) sequenceNr <- decreasingSequenceNrs(emitterDir) if sequenceNr >= lowerSequenceNr } dstFile(emitterDir, sequenceNr).delete() diff --git a/src/multi-jvm/scala/com/rbmhtechnology/eventuate/crdt/ReplicatedORSetSpec.scala b/src/multi-jvm/scala/com/rbmhtechnology/eventuate/crdt/ReplicatedORSetSpec.scala index 0ea9236a..50a5d7d1 100644 --- a/src/multi-jvm/scala/com/rbmhtechnology/eventuate/crdt/ReplicatedORSetSpec.scala +++ b/src/multi-jvm/scala/com/rbmhtechnology/eventuate/crdt/ReplicatedORSetSpec.scala @@ -17,7 +17,6 @@ package com.rbmhtechnology.eventuate.crdt import akka.actor._ -import akka.remote.testconductor.RoleName import akka.remote.testkit._ import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.testkit.TestProbe diff --git a/src/test/scala/com/rbmhtechnology/eventuate/ConditionalRequestsSpec.scala b/src/test/scala/com/rbmhtechnology/eventuate/ConditionalRequestsSpec.scala index b5e9b2e9..db938d03 100644 --- a/src/test/scala/com/rbmhtechnology/eventuate/ConditionalRequestsSpec.scala +++ b/src/test/scala/com/rbmhtechnology/eventuate/ConditionalRequestsSpec.scala @@ -18,6 +18,7 @@ package com.rbmhtechnology.eventuate import akka.actor._ import akka.testkit._ + import com.typesafe.config.ConfigFactory import org.scalatest._