WIP: ResourceEvent-related refactorings
[aquarium] / src / main / scala / gr / grnet / aquarium / user / UserStateComputations.scala
index d59addd..c015377 100644 (file)
@@ -1,5 +1,5 @@
 /*
- * Copyright 2011 GRNET S.A. All rights reserved.
+ * Copyright 2011-2012 GRNET S.A. All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or
  * without modification, are permitted provided that the following
 package gr.grnet.aquarium.user
 
 
-import com.ckkloverdos.maybe.{Failed, NoVal, Just, Maybe}
-import gr.grnet.aquarium.store.{PolicyStore, UserStateStore, ResourceEventStore}
-import gr.grnet.aquarium.util.{ContextualLogger, Loggable, justForSure, failedForSure}
-import gr.grnet.aquarium.logic.accounting.Accounting
-import gr.grnet.aquarium.logic.accounting.algorithm.SimpleCostPolicyAlgorithmCompiler
-import gr.grnet.aquarium.logic.events.{NewWalletEntry, ResourceEvent}
+import scala.collection.mutable
+import gr.grnet.aquarium.util.{ContextualLogger, Loggable}
 import gr.grnet.aquarium.util.date.{TimeHelpers, MutableDateCalc}
-import gr.grnet.aquarium.logic.accounting.dsl.{DSLAgreement, DSLCostPolicy, DSLResourcesMap, DSLPolicy}
+import gr.grnet.aquarium.logic.accounting.dsl.{DSLAgreement, DSLResourcesMap}
+import gr.grnet.aquarium.store.{StoreProvider, PolicyStore}
+import gr.grnet.aquarium.logic.accounting.Accounting
+import gr.grnet.aquarium.logic.accounting.algorithm.CostPolicyAlgorithmCompiler
+import gr.grnet.aquarium.event.NewWalletEntry
+import gr.grnet.aquarium.event.resource.ResourceEventModel
+import gr.grnet.aquarium.event.im.{IMEventModel, StdIMEvent}
+import gr.grnet.aquarium.{AquariumInternalError, AquariumException}
 
 /**
  *
  * @author Christos KK Loverdos <loverdos@gmail.com>
  */
 class UserStateComputations extends Loggable {
-  def createFirstUserState(userId: String,
-                           userCreationMillis: Long,
-                           isActive: Boolean,
-                           credits: Double,
-                           roleNames: List[String] = List(),
-                           agreementName: String = DSLAgreement.DefaultAgreementName) = {
+  def createInitialUserState(imEvent: IMEventModel, credits: Double, agreementName: String) = {
+    if(!imEvent.isCreateUser) {
+      throw new AquariumInternalError(
+        "Got '%s' instead of '%s'".format(imEvent.eventType, IMEventModel.EventTypeNames.create))
+    }
+
+    val userID = imEvent.userID
+    val userCreationMillis = imEvent.occurredMillis
+    val now = TimeHelpers.nowMillis()
+
+    UserState(
+      true,
+      userID,
+      userCreationMillis,
+      0L,
+      false,
+      null,
+      ImplicitlyIssuedResourceEventsSnapshot(List(), now),
+      Nil,
+      Nil,
+      LatestResourceEventsSnapshot(List(), now),
+      0L,
+      0L,
+      IMStateSnapshot(imEvent, now),
+      CreditSnapshot(credits, now),
+      AgreementSnapshot(List(Agreement(agreementName, userCreationMillis)), now),
+      OwnedResourcesSnapshot(Nil, now),
+      Nil,
+      InitialUserStateSetup
+    )
+  }
+
+  def createInitialUserState(userID: String,
+                             userCreationMillis: Long,
+                             isActive: Boolean,
+                             credits: Double,
+                             roleNames: List[String] = List(),
+                             agreementName: String = DSLAgreement.DefaultAgreementName) = {
     val now = userCreationMillis
 
     UserState(
-      userId,
+      true,
+      userID,
       userCreationMillis,
       0L,
       false,
@@ -70,48 +106,64 @@ class UserStateComputations extends Loggable {
       LatestResourceEventsSnapshot(List(), now),
       0L,
       0L,
-      ActiveStateSnapshot(isActive, now),
+      IMStateSnapshot(
+        StdIMEvent(
+          "",
+          now, now, userID,
+          "",
+          isActive, roleNames.headOption.getOrElse("default"),
+          "1.0",
+          IMEventModel.EventTypeNames.create, Map()),
+        now
+      ),
       CreditSnapshot(credits, now),
       AgreementSnapshot(List(Agreement(agreementName, userCreationMillis)), now),
-      RolesSnapshot(roleNames, now),
-      OwnedResourcesSnapshot(Nil, now)
+      OwnedResourcesSnapshot(Nil, now),
+      Nil,
+      InitialUserStateSetup
     )
   }
 
+  def createInitialUserStateFrom(us: UserState): UserState = {
+    createInitialUserState(
+      us.imStateSnapshot.imEvent,
+      us.creditsSnapshot.creditAmount,
+      us.agreementsSnapshot.agreementsByTimeslot.valuesIterator.toList.last)
+  }
+
   def findUserStateAtEndOfBillingMonth(userId: String,
                                        billingMonthInfo: BillingMonthInfo,
-                                       userStateStore: UserStateStore,
-                                       resourceEventStore: ResourceEventStore,
-                                       policyStore: PolicyStore,
-                                       userCreationMillis: Long,
+                                       storeProvider: StoreProvider,
                                        currentUserState: UserState,
-                                       zeroUserState: UserState, 
                                        defaultResourcesMap: DSLResourcesMap,
                                        accounting: Accounting,
-                                       contextualLogger: Maybe[ContextualLogger] = NoVal): Maybe[UserState] = {
+                                       algorithmCompiler: CostPolicyAlgorithmCompiler,
+                                       calculationReason: UserStateChangeReason,
+                                       clogOpt: Option[ContextualLogger] = None): UserState = {
 
     val clog = ContextualLogger.fromOther(
-      contextualLogger,
+      clogOpt,
       logger,
       "findUserStateAtEndOfBillingMonth(%s)", billingMonthInfo)
     clog.begin()
 
-    def doCompute: Maybe[UserState] = {
-      clog.debug("Computing full month billing")
+    def doCompute: UserState = {
       doFullMonthlyBilling(
         userId,
         billingMonthInfo,
-        userStateStore,
-        resourceEventStore,
-        policyStore,
-        userCreationMillis,
+        storeProvider,
         currentUserState,
-        zeroUserState,
         defaultResourcesMap,
         accounting,
-        Just(clog))
+        algorithmCompiler,
+        calculationReason,
+        Some(clog))
     }
 
+    val userStateStore = storeProvider.userStateStore
+    val resourceEventStore = storeProvider.resourceEventStore
+
+    val userCreationMillis = currentUserState.userCreationMillis
     val userCreationDateCalc = new MutableDateCalc(userCreationMillis)
     val billingMonthStartMillis = billingMonthInfo.startMillis
     val billingMonthStopMillis  = billingMonthInfo.stopMillis
@@ -119,330 +171,506 @@ class UserStateComputations extends Loggable {
     if(billingMonthStopMillis < userCreationMillis) {
       // If the user did not exist for this billing month, piece of cake
       clog.debug("User did not exist before %s", userCreationDateCalc)
-      clog.debug("Returning ZERO state %s".format(zeroUserState))
-      clog.endWith(Just(zeroUserState))
+
+      // NOTE: Reason here will be: InitialUserStateSetup$
+      val initialUserState0 = createInitialUserStateFrom(currentUserState)
+      val initialUserState1 = userStateStore.insertUserState(initialUserState0)
+
+      clog.debug("Returning INITIAL state [_id=%s] %s".format(initialUserState1._id, initialUserState1))
+      clog.end()
+
+      initialUserState1
     } else {
       // Ask DB cache for the latest known user state for this billing period
-      val latestUserStateM = userStateStore.findLatestUserStateForEndOfBillingMonth(
+      val latestUserStateOpt = userStateStore.findLatestUserStateForEndOfBillingMonth(
         userId,
         billingMonthInfo.year,
         billingMonthInfo.month)
 
-      latestUserStateM match {
-        case NoVal ⇒
+      latestUserStateOpt match {
+        case None ⇒
           // Not found, must compute
           clog.debug("No user state found from cache, will have to (re)compute")
-          clog.endWith(doCompute)
-          
-        case failed @ Failed(_, _) ⇒
-          clog.warn("Failure while quering cache for user state: %s", failed)
-          clog.endWith(failed)
+          val result = doCompute
+          clog.end()
+          result
 
-        case Just(latestUserState) ⇒
+        case Some(latestUserState) ⇒
           // Found a "latest" user state but need to see if it is indeed the true and one latest.
           // For this reason, we must count the events again.
          val latestStateOOSEventsCounter = latestUserState.billingPeriodOutOfSyncResourceEventsCounter
-         val actualOOSEventsCounterM = resourceEventStore.countOutOfSyncEventsForBillingPeriod(
+         val actualOOSEventsCounter = resourceEventStore.countOutOfSyncEventsForBillingPeriod(
            userId,
            billingMonthStartMillis,
            billingMonthStopMillis)
 
-         actualOOSEventsCounterM match {
-           case NoVal ⇒
-             val errMsg = "No counter computed for out of sync events. Should at least be zero."
+         val counterDiff = actualOOSEventsCounter - latestStateOOSEventsCounter
+         counterDiff match {
+           // ZERO, we are OK!
+           case 0 ⇒
+             // NOTE: Keep the caller's calculation reason
+             latestUserState.copyForChangeReason(calculationReason)
+
+           // We had more, so must recompute
+           case n if n > 0 ⇒
+             clog.debug(
+               "Found %s out of sync events (%s more), will have to (re)compute user state", actualOOSEventsCounter, n)
+             val result = doCompute
+             clog.end()
+             result
+
+           // We had less????
+           case n if n < 0 ⇒
+             val errMsg = "Found %s out of sync events (%s less). DB must be inconsistent".format(actualOOSEventsCounter, n)
              clog.warn(errMsg)
-             clog.endWith(Failed(new Exception(errMsg)))
-
-           case failed @ Failed(_, _) ⇒
-             clog.warn("Failure while querying for out of sync events: %s", failed)
-             clog.endWith(failed)
-
-           case Just(actualOOSEventsCounter) ⇒
-             val counterDiff = actualOOSEventsCounter - latestStateOOSEventsCounter
-             counterDiff match {
-               // ZERO, we are OK!
-               case 0 ⇒
-                 latestUserStateM
-
-               // We had more, so must recompute
-               case n if n > 0 ⇒
-                 clog.debug(
-                   "Found %s out of sync events (%s more), will have to (re)compute user state", actualOOSEventsCounter, n)
-                 clog.endWith(doCompute)
-
-               // We had less????
-               case n if n < 0 ⇒
-                 val errMsg = "Found %s out of sync events (%s less). DB must be inconsistent".format(actualOOSEventsCounter, n)
-                 clog.warn(errMsg)
-                 clog.endWith(Failed(new Exception(errMsg)))
-             }
+             throw new AquariumException(errMsg)
          }
       }
     }
   }
 
+  //+ Utility methods
+  def rcDebugInfo(rcEvent: ResourceEventModel) = {
+    rcEvent.toDebugString(false)
+  }
+  //- Utility methods
+
+  def processResourceEvent(startingUserState: UserState,
+                           userStateWorker: UserStateWorker,
+                           currentResourceEvent: ResourceEventModel,
+                           policyStore: PolicyStore,
+                           stateChangeReason: UserStateChangeReason,
+                           billingMonthInfo: BillingMonthInfo,
+                           walletEntriesBuffer: mutable.Buffer[NewWalletEntry],
+                           algorithmCompiler: CostPolicyAlgorithmCompiler,
+                           clogOpt: Option[ContextualLogger] = None): UserState = {
+
+    val clog = ContextualLogger.fromOther(clogOpt, logger, "walletEntriesForResourceEvent(%s)", currentResourceEvent.id)
+
+    var _workingUserState = startingUserState
+
+    val theResource = currentResourceEvent.safeResource
+    val theInstanceId = currentResourceEvent.safeInstanceId
+    val theValue = currentResourceEvent.value
+
+    val accounting = userStateWorker.accounting
+    val resourcesMap = userStateWorker.resourcesMap
+
+    val currentResourceEventDebugInfo = rcDebugInfo(currentResourceEvent)
+    clog.begin(currentResourceEventDebugInfo)
+
+    userStateWorker.debugTheMaps(clog)(rcDebugInfo)
+
+    // Ignore the event if it is not billable (but still record it in the "previous" stuff).
+    // But to make this decision, first we need the resource definition (and its cost policy).
+    val dslResourceOpt = resourcesMap.findResource(theResource)
+    dslResourceOpt match {
+      // We have a resource (and thus a cost policy)
+      case Some(dslResource) ⇒
+        val costPolicy = dslResource.costPolicy
+        clog.debug("Cost policy %s for %s", costPolicy, dslResource)
+        val isBillable = costPolicy.isBillableEventBasedOnValue(theValue)
+        if(!isBillable) {
+          // The resource event is not billable
+          clog.debug("Ignoring not billable event %s", currentResourceEventDebugInfo)
+        } else {
+          // The resource event is billable
+          // Find the previous event.
+          // This is (potentially) needed to calculate new credit amount and new resource instance amount
+          val previousResourceEventOpt = userStateWorker.findAndRemovePreviousResourceEvent(theResource, theInstanceId)
+          clog.debug("PreviousM %s", previousResourceEventOpt.map(rcDebugInfo(_)))
+
+          val havePreviousResourceEvent = previousResourceEventOpt.isDefined
+          val needPreviousResourceEvent = costPolicy.needsPreviousEventForCreditAndAmountCalculation
+          if(needPreviousResourceEvent && !havePreviousResourceEvent) {
+            // This must be the first resource event of its kind, ever.
+            // TODO: We should normally check the DB to verify the claim (?)
+            clog.debug("Ignoring first event of its kind %s", currentResourceEventDebugInfo)
+            userStateWorker.updateIgnored(currentResourceEvent)
+          } else {
+            val defaultInitialAmount = costPolicy.getResourceInstanceInitialAmount
+            val oldAmount = _workingUserState.getResourceInstanceAmount(theResource, theInstanceId, defaultInitialAmount)
+            val oldCredits = _workingUserState.creditsSnapshot.creditAmount
+
+            // A. Compute new resource instance accumulating amount
+            val newAmount = costPolicy.computeNewAccumulatingAmount(oldAmount, theValue)
+
+            clog.debug("theValue = %s, oldAmount = %s, newAmount = %s, oldCredits = %s", theValue, oldAmount, newAmount, oldCredits)
+
+            // B. Compute new wallet entries
+            clog.debug("agreementsSnapshot = %s", _workingUserState.agreementsSnapshot)
+            val alltimeAgreements = _workingUserState.agreementsSnapshot.agreementsByTimeslot
+
+            //              clog.debug("Computing full chargeslots")
+            val (referenceTimeslot, fullChargeslots) = accounting.computeFullChargeslots(
+              previousResourceEventOpt,
+              currentResourceEvent,
+              oldCredits,
+              oldAmount,
+              newAmount,
+              dslResource,
+              resourcesMap,
+              alltimeAgreements,
+              algorithmCompiler,
+              policyStore,
+              Some(clog)
+            )
+
+            // We have the chargeslots, let's associate them with the current event
+            if(fullChargeslots.length == 0) {
+              // At least one chargeslot is required.
+              throw new AquariumInternalError("No chargeslots computed for resource event %s".format(currentResourceEvent.id))
+            }
+            clog.debugSeq("fullChargeslots", fullChargeslots, 0)
+
+            // C. Compute new credit amount (based on the charge slots)
+            val newCreditsDiff = fullChargeslots.map(_.computedCredits.get).sum
+            val newCredits = oldCredits - newCreditsDiff
+
+            if(stateChangeReason.shouldStoreCalculatedWalletEntries) {
+              val newWalletEntry = NewWalletEntry(
+                userStateWorker.userId,
+                newCreditsDiff,
+                oldCredits,
+                newCredits,
+                TimeHelpers.nowMillis(),
+                referenceTimeslot,
+                billingMonthInfo.year,
+                billingMonthInfo.month,
+                if(havePreviousResourceEvent)
+                  List(currentResourceEvent, previousResourceEventOpt.get)
+                else
+                  List(currentResourceEvent),
+                fullChargeslots,
+                dslResource,
+                currentResourceEvent.isSynthetic
+              )
+              clog.debug("New %s", newWalletEntry)
+
+              walletEntriesBuffer += newWalletEntry
+            } else {
+              clog.debug("newCreditsDiff = %s, newCredits = %s", newCreditsDiff, newCredits)
+            }
+
+            _workingUserState = _workingUserState.copy(
+              creditsSnapshot = CreditSnapshot(newCredits, TimeHelpers.nowMillis()),
+              stateChangeCounter = _workingUserState.stateChangeCounter + 1,
+              totalEventsProcessedCounter = _workingUserState.totalEventsProcessedCounter + 1
+            )
+          }
+        }
+
+        // After processing, all events billable or not update the previous state
+        userStateWorker.updatePrevious(currentResourceEvent)
+
+        _workingUserState = _workingUserState.copy(
+          latestResourceEventsSnapshot = userStateWorker.previousResourceEvents.toImmutableSnapshot(TimeHelpers.nowMillis())
+        )
+
+      // We do not have a resource (and thus, no cost policy)
+      case None ⇒
+        // Now, this is a matter of politics: what do we do if no policy was found?
+        clog.warn("Unknown resource for %s", currentResourceEventDebugInfo)
+    } // dslResourceOpt match
+
+    clog.end(currentResourceEventDebugInfo)
+
+    _workingUserState
+  }
+
+  def processResourceEvents(resourceEvents: Traversable[ResourceEventModel],
+                            startingUserState: UserState,
+                            userStateWorker: UserStateWorker,
+                            policyStore: PolicyStore,
+                            stateChangeReason: UserStateChangeReason,
+                            billingMonthInfo: BillingMonthInfo,
+                            walletEntriesBuffer: mutable.Buffer[NewWalletEntry],
+                            algorithmCompiler: CostPolicyAlgorithmCompiler,
+                            clogOpt: Option[ContextualLogger] = None): UserState = {
+
+    var _workingUserState = startingUserState
+
+    for(currentResourceEvent ← resourceEvents) {
+
+      _workingUserState = processResourceEvent(
+        _workingUserState,
+        userStateWorker,
+        currentResourceEvent,
+        policyStore,
+        stateChangeReason,
+        billingMonthInfo,
+        walletEntriesBuffer,
+        algorithmCompiler,
+        clogOpt
+      )
+    }
+
+    _workingUserState
+  }
+
+
   def doFullMonthlyBilling(userId: String,
                            billingMonthInfo: BillingMonthInfo,
-                           userStateStore: UserStateStore,
-                           resourceEventStore: ResourceEventStore,
-                           policyStore: PolicyStore,
-                           userCreationMillis: Long,
+                           storeProvider: StoreProvider,
                            currentUserState: UserState,
-                           zeroUserState: UserState,
                            defaultResourcesMap: DSLResourcesMap,
                            accounting: Accounting,
-                           contextualLogger: Maybe[ContextualLogger] = NoVal): Maybe[UserState] = Maybe {
+                           algorithmCompiler: CostPolicyAlgorithmCompiler,
+                           calculationReason: UserStateChangeReason = NoSpecificChangeReason,
+                           clogOpt: Option[ContextualLogger] = None): UserState = {
 
-    //+ Utility methods
-    def rcDebugInfo(rcEvent: ResourceEvent) = {
-      rcEvent.toDebugString(defaultResourcesMap, false)
-    }
-    //- Utility methods
 
     val clog = ContextualLogger.fromOther(
-      contextualLogger,
+      clogOpt,
       logger,
       "doFullMonthlyBilling(%s)", billingMonthInfo)
     clog.begin()
 
-    val previousBillingMonthUserStateM = findUserStateAtEndOfBillingMonth(
+    val clogSome = Some(clog)
+
+    val previousBillingMonthUserState = findUserStateAtEndOfBillingMonth(
       userId,
       billingMonthInfo.previousMonth,
-      userStateStore,
-      resourceEventStore,
-      policyStore,
-      userCreationMillis,
+      storeProvider,
       currentUserState,
-      zeroUserState,
       defaultResourcesMap,
       accounting,
-      Just(clog)
+      algorithmCompiler,
+      calculationReason.forPreviousBillingMonth,
+      clogSome
     )
-    
-    if(previousBillingMonthUserStateM.isNoVal) {
-      throw new Exception("Could not calculate initial user state for billing %s".format(billingMonthInfo))
-    }
-    if(previousBillingMonthUserStateM.isFailed) {
-      throw failedForSure(previousBillingMonthUserStateM).exception
-    }
+
+    val startingUserState = previousBillingMonthUserState
+
+    val userStateStore = storeProvider.userStateStore
+    val resourceEventStore = storeProvider.resourceEventStore
+    val policyStore = storeProvider.policyStore
 
     val billingMonthStartMillis = billingMonthInfo.startMillis
     val billingMonthEndMillis = billingMonthInfo.stopMillis
 
-    val startingUserState = justForSure(previousBillingMonthUserStateM).get
     // Keep the working (current) user state. This will get updated as we proceed with billing for the month
     // specified in the parameters.
-    var _workingUserState = startingUserState
-
-    // This is a collection of all the latest resource events.
-    // We want these in order to correlate incoming resource events with their previous (in `occurredMillis` time)
-    // ones.
-    // Will be updated on processing the next resource event.
-    val previousResourceEvents = startingUserState.latestResourceEventsSnapshot.toMutableWorker
-    // Prepare the implicitly terminated resource events from previous billing period
-    val implicitlyTerminatedResourceEvents = _workingUserState.implicitlyTerminatedSnapshot.toMutableWorker
-    // Keep the resource events from this period that were first (and unused) of their kind
-    val ignoredFirstResourceEvents = IgnoredFirstResourceEventsWorker.Empty
-
-    /**
-     * Finds the previous resource event by checking two possible sources: a) The implicitly terminated resource
-     * events and b) the explicit previous resource events. If the event is found, it is removed from the
-     * respective source.
-     *
-     * If the event is not found, then this must be for a new resource instance.
-     * (and probably then some `zero` resource event must be implied as the previous one)
-     *
-     * @param resource
-     * @param instanceId
-     * @return
-     */
-    def findAndRemovePreviousResourceEvent(resource: String, instanceId: String): Maybe[ResourceEvent] = {
-      // implicitly terminated events are checked first
-      implicitlyTerminatedResourceEvents.findAndRemoveResourceEvent(resource, instanceId) match {
-        case just @ Just(_) ⇒
-          just
-        case NoVal ⇒
-          // explicit previous resource events are checked second
-          previousResourceEvents.findAndRemoveResourceEvent(resource, instanceId) match {
-            case just @ Just(_) ⇒
-              just
-            case noValOrFailed ⇒
-              noValOrFailed
-          }
-        case failed ⇒
-          failed
-      }
-    }
+    // NOTE: The calculation reason is not the one we get from the previous user state but the one our caller specifies
+    var _workingUserState = startingUserState.copyForChangeReason(calculationReason)
 
-    def debugTheMaps(): Unit = {
-      if(previousResourceEvents.size > 0) {
-        val map = previousResourceEvents.resourceEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
-        clog.debugMap("previousResourceEvents", map, 0)
-      }
-      if(implicitlyTerminatedResourceEvents.size > 0) {
-        val map = implicitlyTerminatedResourceEvents.implicitlyIssuedEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
-        clog.debug("implicitlyTerminatedResourceEvents", map, 0)
-      }
-      if(ignoredFirstResourceEvents.size > 0) {
-        val map = ignoredFirstResourceEvents.ignoredFirstEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
-        clog.debug("%s ignoredFirstResourceEvents", map, 0)
-      }
-    }
+    val userStateWorker = UserStateWorker.fromUserState(_workingUserState, accounting, defaultResourcesMap)
 
-    debugTheMaps()
+    userStateWorker.debugTheMaps(clog)(rcDebugInfo)
 
-    // Find the actual resource events from DB
+    // First, find and process the actual resource events from DB
     val allResourceEventsForMonth = resourceEventStore.findAllRelevantResourceEventsForBillingPeriod(
       userId,
       billingMonthStartMillis,
       billingMonthEndMillis)
-    var _eventCounter = 0
 
-    clog.debug("resourceEventStore = %s".format(resourceEventStore))
-    if(allResourceEventsForMonth.size > 0) {
-      clog.debug("Found %s resource events, starting processing...", allResourceEventsForMonth.size)
-    } else {
-      clog.debug("Not found any resource events")
+    val newWalletEntries = scala.collection.mutable.ListBuffer[NewWalletEntry]()
+
+    _workingUserState = processResourceEvents(
+      allResourceEventsForMonth,
+      _workingUserState,
+      userStateWorker,
+      policyStore,
+      calculationReason,
+      billingMonthInfo,
+      newWalletEntries,
+      algorithmCompiler,
+      clogSome
+    )
+
+    // Second, for the remaining events which must contribute an implicit OFF, we collect those OFFs
+    // ... in order to generate an implicit ON later
+    val (specialEvents, theirImplicitEnds) = userStateWorker.
+      findAndRemoveGeneratorsOfImplicitEndEvents(billingMonthEndMillis)
+    if(specialEvents.lengthCompare(1) >= 0 || theirImplicitEnds.lengthCompare(1) >= 0) {
+      clog.debug("")
+      clog.debug("Process implicitly issued events")
+      clog.debugSeq("specialEvents", specialEvents, 0)
+      clog.debugSeq("theirImplicitEnds", theirImplicitEnds, 0)
+    }
+
+    // Now, the previous and implicitly started must be our base for the following computation, so we create an
+    // appropriate worker
+    val specialUserStateWorker = UserStateWorker(
+      userStateWorker.userId,
+      LatestResourceEventsWorker.fromList(specialEvents),
+      ImplicitlyIssuedResourceEventsWorker.Empty,
+      IgnoredFirstResourceEventsWorker.Empty,
+      userStateWorker.accounting,
+      userStateWorker.resourcesMap
+    )
+
+    _workingUserState = processResourceEvents(
+      theirImplicitEnds,
+      _workingUserState,
+      specialUserStateWorker,
+      policyStore,
+      calculationReason,
+      billingMonthInfo,
+      newWalletEntries,
+      algorithmCompiler,
+      clogSome
+    )
+
+    val lastUpdateTime = TimeHelpers.nowMillis()
+
+    _workingUserState = _workingUserState.copy(
+      implicitlyIssuedSnapshot = userStateWorker.implicitlyIssuedStartEvents.toImmutableSnapshot(lastUpdateTime),
+      latestResourceEventsSnapshot = userStateWorker.previousResourceEvents.toImmutableSnapshot(lastUpdateTime),
+      stateChangeCounter = _workingUserState.stateChangeCounter + 1,
+      parentUserStateId = startingUserState.idOpt,
+      newWalletEntries = newWalletEntries.toList
+    )
+
+    clog.debug("calculationReason = %s", calculationReason)
+
+    if(calculationReason.shouldStoreUserState) {
+      val storedUserState = userStateStore.insertUserState(_workingUserState)
+      clog.debug("Saved [_id=%s] %s", storedUserState._id, storedUserState)
+      _workingUserState = storedUserState
     }
 
-    for {
-      currentResourceEvent <- allResourceEventsForMonth
-    } {
-      _eventCounter = _eventCounter + 1
-      
-      if(_eventCounter == 1) {
-        clog.debugMap("defaultResourcesMap", defaultResourcesMap.map, 1)
-      } else {
-        clog.debug("")
+    clog.debug("RETURN %s", _workingUserState)
+    clog.end()
+    _workingUserState
+  }
+}
+
+/**
+ * A helper object holding intermediate state/results during resource event processing.
+ *
+ * @param previousResourceEvents
+ *          This is a collection of all the latest resource events.
+ *          We want these in order to correlate incoming resource events with their previous (in `occurredMillis` time)
+ *          ones. Will be updated on processing the next resource event.
+ *
+ * @param implicitlyIssuedStartEvents
+ *          The implicitly issued resource events at the beginning of the billing period.
+ *
+ * @param ignoredFirstResourceEvents
+ *          The resource events that were first (and unused) of their kind.
+ *
+ * @author Christos KK Loverdos <loverdos@gmail.com>
+ */
+case class UserStateWorker(userId: String,
+                           previousResourceEvents: LatestResourceEventsWorker,
+                           implicitlyIssuedStartEvents: ImplicitlyIssuedResourceEventsWorker,
+                           ignoredFirstResourceEvents: IgnoredFirstResourceEventsWorker,
+                           accounting: Accounting,
+                           resourcesMap: DSLResourcesMap) {
+
+  /**
+   * Finds the previous resource event by checking two possible sources: a) The implicitly terminated resource
+   * events and b) the explicit previous resource events. If the event is found, it is removed from the
+   * respective source.
+   *
+   * If the event is not found, then this must be for a new resource instance.
+   * (and probably then some `zero` resource event must be implied as the previous one)
+   *
+   * @param resource
+   * @param instanceId
+   * @return
+   */
+  def findAndRemovePreviousResourceEvent(resource: String, instanceId: String): Option[ResourceEventModel] = {
+    // implicitly issued events are checked first
+    implicitlyIssuedStartEvents.findAndRemoveResourceEvent(resource, instanceId) match {
+      case some @ Some(_) ⇒
+        some
+      case None ⇒
+        // explicit previous resource events are checked second
+        previousResourceEvents.findAndRemoveResourceEvent(resource, instanceId) match {
+          case some @ Some(_) ⇒
+            some
+          case _ ⇒
+            None
+        }
+    }
+  }
+
+  def updateIgnored(resourceEvent: ResourceEventModel): Unit = {
+    ignoredFirstResourceEvents.updateResourceEvent(resourceEvent)
+  }
+
+  def updatePrevious(resourceEvent: ResourceEventModel): Unit = {
+    previousResourceEvents.updateResourceEvent(resourceEvent)
+  }
+
+  def debugTheMaps(clog: ContextualLogger)(rcDebugInfo: ResourceEventModel ⇒ String): Unit = {
+    if(previousResourceEvents.size > 0) {
+      val map = previousResourceEvents.latestEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
+      clog.debugMap("previousResourceEvents", map, 0)
+    }
+    if(implicitlyIssuedStartEvents.size > 0) {
+      val map = implicitlyIssuedStartEvents.implicitlyIssuedEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
+      clog.debugMap("implicitlyTerminatedResourceEvents", map, 0)
+    }
+    if(ignoredFirstResourceEvents.size > 0) {
+      val map = ignoredFirstResourceEvents.ignoredFirstEventsMap.map { case (k, v) => (k, rcDebugInfo(v)) }
+      clog.debugMap("ignoredFirstResourceEvents", map, 0)
+    }
+  }
+
+//  private[this]
+//  def allPreviousAndAllImplicitlyStarted: List[ResourceEvent] = {
+//    val buffer: FullMutableResourceTypeMap = scala.collection.mutable.Map[FullResourceType, ResourceEvent]()
+//
+//    buffer ++= implicitlyIssuedStartEvents.implicitlyIssuedEventsMap
+//    buffer ++= previousResourceEvents.latestEventsMap
+//
+//    buffer.valuesIterator.toList
+//  }
+
+  /**
+   * Find those events from `implicitlyIssuedStartEvents` and `previousResourceEvents` that will generate implicit
+   * end events along with those implicitly issued events. Before returning, remove the events that generated the
+   * implicit ends from the internal state of this instance.
+   *
+   * @see [[gr.grnet.aquarium.logic.accounting.dsl.DSLCostPolicy]]
+   */
+  def findAndRemoveGeneratorsOfImplicitEndEvents(newOccuredMillis: Long
+                                                ): (List[ResourceEventModel], List[ResourceEventModel]) = {
+    val buffer = mutable.ListBuffer[(ResourceEventModel, ResourceEventModel)]()
+    val checkSet = mutable.Set[ResourceEventModel]()
+
+    def doItFor(map: ResourceEventModel.FullMutableResourceTypeMap): Unit = {
+      val resourceEvents = map.valuesIterator
+      for {
+        resourceEvent ← resourceEvents
+        dslResource   ← resourcesMap.findResource(resourceEvent.safeResource)
+        costPolicy    =  dslResource.costPolicy
+      } {
+        if(costPolicy.supportsImplicitEvents) {
+          if(costPolicy.mustConstructImplicitEndEventFor(resourceEvent)) {
+            val implicitEnd = costPolicy.constructImplicitEndEventFor(resourceEvent, newOccuredMillis)
+
+            if(!checkSet.contains(resourceEvent)) {
+              checkSet.add(resourceEvent)
+              buffer append ((resourceEvent, implicitEnd))
+            }
+
+            // remove it anyway
+            map.remove((resourceEvent.safeResource, resourceEvent.safeInstanceId))
+          }
+        }
       }
-      
-      val theResource = currentResourceEvent.safeResource
-      val theInstanceId = currentResourceEvent.safeInstanceId
-      val theValue = currentResourceEvent.value
-
-      clog.debug("Processing %s", currentResourceEvent)
-      val currentResourceEventDebugInfo = rcDebugInfo(currentResourceEvent)
-      clog.begin(currentResourceEventDebugInfo)
-
-      debugTheMaps()
-
-      // Ignore the event if it is not billable (but still record it in the "previous" stuff).
-      // But to make this decision, first we need the resource definition (and its cost policy).
-      val dslResourceOpt = defaultResourcesMap.findResource(theResource)
-
-      dslResourceOpt match {
-        // We have a resource (and thus a cost policy)
-        case Some(dslResource) ⇒
-          val costPolicy = dslResource.costPolicy
-          clog.debug("Cost policy %s for %s", costPolicy, dslResource)
-          val isBillable = costPolicy.isBillableEventBasedOnValue(theValue)
-          isBillable match {
-            // The resource event is not billable
-            case false ⇒
-              clog.debug("Ignoring not billable event %s", currentResourceEventDebugInfo)
-
-            // The resource event is billable
-            case true ⇒
-              // Find the previous event.
-              // This is (potentially) needed to calculate new credit amount and new resource instance amount
-              val previousResourceEventM = findAndRemovePreviousResourceEvent(theResource, theInstanceId)
-              clog.debug("PreviousM %s", previousResourceEventM.map(rcDebugInfo(_)))
-
-              val havePreviousResourceEvent = previousResourceEventM.isJust
-              val needPreviousResourceEvent = costPolicy.needsPreviousEventForCreditAndAmountCalculation
-              if(needPreviousResourceEvent && !havePreviousResourceEvent) {
-                // This must be the first resource event of its kind, ever.
-                // TODO: We should normally check the DB to verify the claim (?)
-                clog.info("Ignoring first event of its kind %s", currentResourceEventDebugInfo)
-                ignoredFirstResourceEvents.updateResourceEvent(currentResourceEvent)
-              } else {
-                val defaultInitialAmount = costPolicy.getResourceInstanceInitialAmount
-                val oldAmount = _workingUserState.getResourceInstanceAmount(theResource, theInstanceId, defaultInitialAmount)
-                val oldCredits = _workingUserState.creditsSnapshot.creditAmount
-
-                // A. Compute new resource instance accumulating amount
-                val newAmount = costPolicy.computeNewAccumulatingAmount(oldAmount, theValue)
-
-                clog.debug("theValue = %s, oldAmount = %s, newAmount = %s, oldCredits = %s", theValue, oldAmount, newAmount, oldCredits)
-
-                // B. Compute new wallet entries
-                clog.debug("agreementsSnapshot = %s", _workingUserState.agreementsSnapshot)
-                val alltimeAgreements = _workingUserState.agreementsSnapshot.agreementsByTimeslot
-
-                clog.debug("Computing full chargeslots")
-                val fullChargeslotsM = accounting.computeFullChargeslots(
-                  previousResourceEventM,
-                  currentResourceEvent,
-                  oldCredits,
-                  oldAmount,
-                  newAmount,
-                  dslResource,
-                  defaultResourcesMap,
-                  alltimeAgreements,
-                  SimpleCostPolicyAlgorithmCompiler,
-                  policyStore,
-                  Just(clog)
-                )
-
-                // We have the chargeslots, let's associate them with the current event
-                fullChargeslotsM match {
-                  case Just(fullChargeslots) ⇒
-                    if(fullChargeslots.length == 0) {
-                      // At least one chargeslot is required.
-                      throw new Exception("No chargeslots computed for resource event %s".format(currentResourceEvent.id))
-                    }
-                    clog.debugSeq("fullChargeslots", fullChargeslots, 1)
-
-                    // C. Compute new credit amount (based on the charge slots)
-                    val newCreditsDiff = fullChargeslots.map(_.computedCredits.get).sum
-                    val newCredits = oldCredits - newCreditsDiff
-                    clog.debug("newCreditsDiff = %s, newCredits = %s", newCreditsDiff, newCredits)
-
-                    val newWalletEntry = NewWalletEntry(
-                      userId,
-                      newCreditsDiff,
-                      oldCredits,
-                      newCredits,
-                      TimeHelpers.nowMillis,
-                      billingMonthInfo.year,
-                      billingMonthInfo.month,
-                      currentResourceEvent,
-                      previousResourceEventM.toOption,
-                      fullChargeslots,
-                      dslResource
-                    )
-
-                    clog.debug("New %s", newWalletEntry)
-
-                    _workingUserState = _workingUserState.copy(
-                      creditsSnapshot = CreditSnapshot(newCredits, TimeHelpers.nowMillis),
-                      stateChangeCounter = _workingUserState.stateChangeCounter + 1,
-                      totalEventsProcessedCounter = _workingUserState.totalEventsProcessedCounter + 1
-                    )
-
-                  case NoVal ⇒
-                    // At least one chargeslot is required.
-                    throw new Exception("No chargeslots computed")
-
-                  case failed @ Failed(e, m) ⇒
-                    throw new Exception(m, e)
-                }
-              }
-
-          } // isBillable
-
-          // After processing, all event, billable or not update the previous state
-          previousResourceEvents.updateResourceEvent(currentResourceEvent)
-
-        // We do not have a resource (and no cost policy)
-        case None ⇒
-          // Now, this is a matter of politics: what do we do if no policy was found?
-          clog.warn("No cost policy for %s", currentResourceEventDebugInfo)
-      } // dslResourceOpt match
+    }
 
-      clog.end(currentResourceEventDebugInfo)
-    } // for { currentResourceEvent <- allResourceEventsForMonth }
+    doItFor(previousResourceEvents.latestEventsMap)                // we give priority for previous
+    doItFor(implicitlyIssuedStartEvents.implicitlyIssuedEventsMap) // ... over implicitly issued...
 
-    clog.endWith(_workingUserState)
+    (buffer.view.map(_._1).toList, buffer.view.map(_._2).toList)
+  }
+}
+
+object UserStateWorker {
+  def fromUserState(userState: UserState, accounting: Accounting, resourcesMap: DSLResourcesMap): UserStateWorker = {
+    UserStateWorker(
+      userState.userID,
+      userState.latestResourceEventsSnapshot.toMutableWorker,
+      userState.implicitlyIssuedSnapshot.toMutableWorker,
+      IgnoredFirstResourceEventsWorker.Empty,
+      accounting,
+      resourcesMap
+    )
   }
 }