WIP Resource event handling
[aquarium] / src / main / scala / gr / grnet / aquarium / store / mongodb / MongoDBStore.scala
index 86e73e5..04ecf59 100644 (file)
 package gr.grnet.aquarium.store.mongodb
 
 import com.mongodb.util.JSON
-import gr.grnet.aquarium.user.UserState
-import gr.grnet.aquarium.user.UserState.{JsonNames ⇒ UserStateJsonNames}
+import gr.grnet.aquarium.computation.state.UserState.{JsonNames ⇒ UserStateJsonNames}
 import gr.grnet.aquarium.util.json.JsonSupport
 import collection.mutable.ListBuffer
-import gr.grnet.aquarium.event._
-import gr.grnet.aquarium.event.im.IMEventModel
-import gr.grnet.aquarium.event.im.IMEventModel.{Names ⇒ IMEventNames}
-import gr.grnet.aquarium.event.resource.ResourceEventModel
-import gr.grnet.aquarium.event.resource.ResourceEventModel.{Names ⇒ ResourceEventNames}
+import gr.grnet.aquarium.event.model.im.IMEventModel
+import gr.grnet.aquarium.event.model.im.IMEventModel.{Names ⇒ IMEventNames}
+import gr.grnet.aquarium.event.model.resource.ResourceEventModel
+import gr.grnet.aquarium.event.model.resource.ResourceEventModel.{Names ⇒ ResourceEventNames}
 import gr.grnet.aquarium.store._
-import gr.grnet.aquarium.event.WalletEntry.{JsonNames ⇒ WalletJsonNames}
-import gr.grnet.aquarium.event.PolicyEntry.{JsonNames ⇒ PolicyJsonNames}
-import java.util.Date
+import gr.grnet.aquarium.event.model.PolicyEntry.{JsonNames ⇒ PolicyJsonNames}
 import gr.grnet.aquarium.logic.accounting.Policy
 import com.mongodb._
 import org.bson.types.ObjectId
 import com.ckkloverdos.maybe.Maybe
 import gr.grnet.aquarium.util._
 import gr.grnet.aquarium.converter.Conversions
+import gr.grnet.aquarium.computation.state.UserState
+import gr.grnet.aquarium.event.model.{ExternalEventModel, PolicyEntry}
+import gr.grnet.aquarium.computation.BillingMonthInfo
+import gr.grnet.aquarium.Aquarium
 
 /**
  * Mongodb implementation of the various aquarium stores.
@@ -69,7 +69,6 @@ class MongoDBStore(
     val password: String)
   extends ResourceEventStore
   with UserStateStore
-  with WalletEntryStore
   with IMEventStore
   with PolicyStore
   with Loggable {
@@ -81,7 +80,6 @@ class MongoDBStore(
   private[store] lazy val userStates       = getCollection(MongoDBStore.USER_STATES_COLLECTION)
   private[store] lazy val imEvents         = getCollection(MongoDBStore.IM_EVENTS_COLLECTION)
   private[store] lazy val unparsedIMEvents = getCollection(MongoDBStore.UNPARSED_IM_EVENTS_COLLECTION)
-  private[store] lazy val walletEntries    = getCollection(MongoDBStore.WALLET_ENTRIES_COLLECTION)
   private[store] lazy val policyEntries    = getCollection(MongoDBStore.POLICY_ENTRIES_COLLECTION)
 
   private[this] def getCollection(name: String): DBCollection = {
@@ -104,8 +102,12 @@ class MongoDBStore(
     MongoDBResourceEvent.fromOther(event, null)
   }
 
+  def pingResourceEventStore(): Unit = synchronized {
+    MongoDBStore.ping(mongo)
+  }
+
   def insertResourceEvent(event: ResourceEventModel) = {
-    val localEvent = MongoDBResourceEvent.fromOther(event, new ObjectId())
+    val localEvent = MongoDBResourceEvent.fromOther(event, new ObjectId().toStringMongod)
     MongoDBStore.insertObject(localEvent, resourceEvents, MongoDBStore.jsonSupportToDBObject)
     localEvent
   }
@@ -184,11 +186,22 @@ class MongoDBStore(
     MongoDBStore.runQuery[ResourceEvent](query, resourceEvents, orderBy)(MongoDBResourceEvent.fromDBObject)(None)
   }
   
-  def countOutOfSyncEventsForBillingPeriod(userId: String, startMillis: Long, stopMillis: Long): Maybe[Long] = {
-    Maybe {
-      // FIXME: Implement
-      0L
-    }
+  def countOutOfSyncResourceEventsForBillingPeriod(userID: String, startMillis: Long, stopMillis: Long): Long = {
+    val query = new BasicDBObjectBuilder().
+      add(ResourceEventModel.Names.userID, userID).
+      // received within the period
+      add(ResourceEventModel.Names.receivedMillis, new BasicDBObject("$gte", startMillis)).
+      add(ResourceEventModel.Names.receivedMillis, new BasicDBObject("$lte", stopMillis)).
+      // occurred outside the period
+      add("$or", {
+        val dbList = new BasicDBList()
+        dbList.add(0, new BasicDBObject(ResourceEventModel.Names.occurredMillis, new BasicDBObject("$lt", startMillis)))
+        dbList.add(1, new BasicDBObject(ResourceEventModel.Names.occurredMillis, new BasicDBObject("$gt", stopMillis)))
+        dbList
+      }).
+      get()
+
+    resourceEvents.count(query)
   }
 
   def findAllRelevantResourceEventsForBillingPeriod(userId: String,
@@ -201,148 +214,125 @@ class MongoDBStore(
 
   //+ UserStateStore
   def insertUserState(userState: UserState) = {
-    MongoDBStore.insertUserState(userState, userStates, MongoDBStore.jsonSupportToDBObject)
+    MongoDBStore.insertUserState(
+      userState.copy(_id = new ObjectId().toString),
+      userStates,
+      MongoDBStore.jsonSupportToDBObject
+    )
   }
 
   def findUserStateByUserID(userID: String): Option[UserState] = {
     val query = new BasicDBObject(UserStateJsonNames.userID, userID)
     val cursor = userStates find query
 
-    withCloseable(cursor) { cursor ⇒
-      if(cursor.hasNext)
-        Some(MongoDBStore.dbObjectToUserState(cursor.next()))
-      else
-        None
-    }
+    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
   }
 
+  def findLatestUserStateForFullMonthBilling(userID: String, bmi: BillingMonthInfo): Option[UserState] = {
+    val query = new BasicDBObjectBuilder().
+      add(UserState.JsonNames.userID, userID).
+      add(UserState.JsonNames.isFullBillingMonthState, true).
+      add(UserState.JsonNames.theFullBillingMonth_year, bmi.year).
+      add(UserState.JsonNames.theFullBillingMonth_month, bmi.month).
+      get()
 
-  def findLatestUserStateByUserID(userID: String) = {
-    // FIXME: implement
-    null
-  }
+    logger.debug("findLatestUserStateForFullMonthBilling(%s, %s) query: %s".format(userID, bmi, query))
 
-  def findLatestUserStateForEndOfBillingMonth(userId: String,
-                                              yearOfBillingMonth: Int,
-                                              billingMonth: Int): Option[UserState] = {
-    None // FIXME: implement
-  }
+    // Descending order, so that the latest comes first
+    val sorter = new BasicDBObject(UserState.JsonNames.occurredMillis, -1)
+
+    val cursor = userStates.find(query).sort(sorter)
 
-  def deleteUserState(userId: String) = {
-    val query = new BasicDBObject(UserStateJsonNames.userID, userId)
-    userStates.findAndRemove(query)
+    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
   }
   //- UserStateStore
 
-  //+WalletEntryStore
-  def storeWalletEntry(entry: WalletEntry): Maybe[RecordID] = {
-    Maybe {
-      MongoDBStore.storeAny[WalletEntry](
-        entry,
-        walletEntries,
-        WalletJsonNames.id,
-        (e) => e.id,
-        MongoDBStore.jsonSupportToDBObject)
-    }
+  //+IMEventStore
+  def createIMEventFromJson(json: String) = {
+    MongoDBStore.createIMEventFromJson(json)
   }
 
-  def findWalletEntryById(id: String): Maybe[WalletEntry] = {
-    MongoDBStore.findBy(WalletJsonNames.id, id, walletEntries, MongoDBStore.dbObjectToWalletEntry): Maybe[WalletEntry]
+  def createIMEventFromOther(event: IMEventModel) = {
+    MongoDBStore.createIMEventFromOther(event)
   }
 
-  def findUserWalletEntries(userId: String) = {
-    // TODO: optimize
-    findUserWalletEntriesFromTo(userId, new Date(0), new Date(Int.MaxValue))
+  def pingIMEventStore(): Unit = {
+    MongoDBStore.ping(mongo)
   }
 
-  def findUserWalletEntriesFromTo(userId: String, from: Date, to: Date) : List[WalletEntry] = {
-    val q = new BasicDBObject()
-    // TODO: Is this the correct way for an AND query?
-    q.put(WalletJsonNames.occurredMillis, new BasicDBObject("$gt", from.getTime))
-    q.put(WalletJsonNames.occurredMillis, new BasicDBObject("$lt", to.getTime))
-    q.put(WalletJsonNames.userId, userId)
-
-    MongoDBStore.runQuery[WalletEntry](q, walletEntries)(MongoDBStore.dbObjectToWalletEntry)(Some(_sortByTimestampAsc))
+  def insertIMEvent(event: IMEventModel): IMEvent = {
+    val localEvent = MongoDBIMEvent.fromOther(event, new ObjectId().toStringMongod)
+    MongoDBStore.insertObject(localEvent, imEvents, MongoDBStore.jsonSupportToDBObject)
+    localEvent
   }
 
-  def findWalletEntriesAfter(userId: String, from: Date) : List[WalletEntry] = {
-    val q = new BasicDBObject()
-    q.put(WalletJsonNames.occurredMillis, new BasicDBObject("$gt", from.getTime))
-    q.put(WalletJsonNames.userId, userId)
-
-    MongoDBStore.runQuery[WalletEntry](q, walletEntries)(MongoDBStore.dbObjectToWalletEntry)(Some(_sortByTimestampAsc))
+  def findIMEventById(id: String): Option[IMEvent] = {
+    MongoDBStore.findBy(IMEventNames.id, id, imEvents, MongoDBIMEvent.fromDBObject)
   }
 
-  def findLatestUserWalletEntries(userId: String) = {
-    Maybe {
-      val orderBy = new BasicDBObject(WalletJsonNames.occurredMillis, -1) // -1 is descending order
-      val cursor = walletEntries.find().sort(orderBy)
-
-      try {
-        val buffer = new scala.collection.mutable.ListBuffer[WalletEntry]
-        if(cursor.hasNext) {
-          val walletEntry = MongoDBStore.dbObjectToWalletEntry(cursor.next())
-          buffer += walletEntry
 
-          var _previousOccurredMillis = walletEntry.occurredMillis
-          var _ok = true
+  /**
+   * Find the `CREATE` even for the given user. Note that there must be only one such event.
+   */
+  def findCreateIMEventByUserID(userID: String): Option[IMEvent] = {
+    val query = new BasicDBObjectBuilder().
+      add(IMEventNames.userID, userID).
+      add(IMEventNames.eventType, IMEventModel.EventTypeNames.create).get()
 
-          while(cursor.hasNext && _ok) {
-            val walletEntry = MongoDBStore.dbObjectToWalletEntry(cursor.next())
-            var currentOccurredMillis = walletEntry.occurredMillis
-            _ok = currentOccurredMillis == _previousOccurredMillis
-            
-            if(_ok) {
-              buffer += walletEntry
-            }
-          }
+    // Normally one such event is allowed ...
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
 
-          buffer.toList
-        } else {
-          null
-        }
-      } finally {
-        cursor.close()
-      }
-    }
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
   }
 
-  def findPreviousEntry(userId: String, resource: String,
-                        instanceId: String,
-                        finalized: Option[Boolean]): List[WalletEntry] = {
-    val q = new BasicDBObject()
-    q.put(WalletJsonNames.userId, userId)
-    q.put(WalletJsonNames.resource, resource)
-    q.put(WalletJsonNames.instanceId, instanceId)
-    finalized match {
-      case Some(x) => q.put(WalletJsonNames.finalized, x)
-      case None =>
-    }
+  def findLatestIMEventByUserID(userID: String): Option[IMEvent] = {
+    val query = new BasicDBObject(IMEventNames.userID, userID)
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, -1))
 
-    MongoDBStore.runQuery[WalletEntry](q, walletEntries)(MongoDBStore.dbObjectToWalletEntry)(Some(_sortByTimestampAsc))
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
   }
-  //-WalletEntryStore
 
-  //+IMEventStore
-  def createIMEventFromJson(json: String) = {
-    MongoDBStore.createIMEventFromJson(json)
-  }
+  /**
+   * Find the very first activation event for a particular user.
+   *
+   */
+  def findFirstIsActiveIMEventByUserID(userID: String): Option[IMEvent] = {
+    val query = new BasicDBObjectBuilder().
+      add(IMEventNames.userID, userID).
+      add(IMEventNames.isActive, true).get()
 
-  def createIMEventFromOther(event: IMEventModel) = {
-    MongoDBStore.createIMEventFromOther(event)
-  }
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
 
-  def insertIMEvent(event: IMEventModel): IMEvent = {
-    val localEvent = MongoDBIMEvent.fromOther(event, new ObjectId())
-    MongoDBStore.insertObject(localEvent, imEvents, MongoDBStore.jsonSupportToDBObject)
-    localEvent
+    withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext) {
+        Some(MongoDBIMEvent.fromDBObject(cursor.next()))
+      } else {
+        None
+      }
+   }
   }
 
-  def findIMEventById(id: String): Option[IMEvent] = {
-    MongoDBStore.findBy(IMEventNames.id, id, imEvents, MongoDBIMEvent.fromDBObject)
+  /**
+   * Scans events for the given user, sorted by `occurredMillis` in ascending order and runs them through
+   * the given function `f`.
+   *
+   * Any exception is propagated to the caller. The underlying DB resources are properly disposed in any case.
+   */
+  def replayIMEventsInOccurrenceOrder(userID: String)(f: (IMEvent) => Unit) = {
+    val query = new BasicDBObject(IMEventNames.userID, userID)
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
+
+    withCloseable(cursor) { cursor ⇒
+      while(cursor.hasNext) {
+        val model = MongoDBIMEvent.fromDBObject(cursor.next())
+        f(model)
+      }
+    }
   }
   //-IMEventStore
 
+
+
   //+PolicyStore
   def loadPolicyEntriesAfter(after: Long): List[PolicyEntry] = {
     val query = new BasicDBObject(PolicyEntry.JsonNames.validFrom,
@@ -373,28 +363,29 @@ object MongoDBStore {
   }
 
   /**
-   * Collection holding the [[gr.grnet.aquarium.event.ResourceEvent]]s.
+   * Collection holding the [[gr.grnet.aquarium.event.model.resource.ResourceEventModel]]s.
    *
    * Resource events are coming from all systems handling billable resources.
    */
   final val RESOURCE_EVENTS_COLLECTION = "resevents"
 
   /**
-   * Collection holding the snapshots of [[gr.grnet.aquarium.user.UserState]].
+   * Collection holding the snapshots of [[gr.grnet.aquarium.computation.state.UserState]].
    *
-   * [[gr.grnet.aquarium.user.UserState]] is held internally within [[gr.grnet.aquarium.actor.service.user .UserActor]]s.
+   * [[gr.grnet.aquarium.computation.state.UserState]] is held internally within
+   * [[gr.grnet.aquarium.actor.service.user .UserActor]]s.
    */
   final val USER_STATES_COLLECTION = "userstates"
 
   /**
-   * Collection holding [[gr.grnet.aquarium.event.im.IMEventModel]]s.
+   * Collection holding [[gr.grnet.aquarium.event.model.im.IMEventModel]]s.
    *
    * User events are coming from the IM module (external).
    */
   final val IM_EVENTS_COLLECTION = "imevents"
 
   /**
-   * Collection holding [[gr.grnet.aquarium.event.im.IMEventModel]]s that could not be parsed to normal objects.
+   * Collection holding [[gr.grnet.aquarium.event.model.im.IMEventModel]]s that could not be parsed to normal objects.
    *
    * We of course assume at least a valid JSON representation.
    *
@@ -403,19 +394,12 @@ object MongoDBStore {
   final val UNPARSED_IM_EVENTS_COLLECTION = "unparsed_imevents"
 
   /**
-   * Collection holding [[gr.grnet.aquarium.event.WalletEntry]].
-   *
-   * Wallet entries are generated internally in Aquarium.
-   */
-  final val WALLET_ENTRIES_COLLECTION = "wallets"
-
-  /**
    * Collection holding [[gr.grnet.aquarium.logic.accounting.dsl.DSLPolicy]].
    */
 //  final val POLICIES_COLLECTION = "policies"
 
   /**
-   * Collection holding [[gr.grnet.aquarium.event.PolicyEntry]].
+   * Collection holding [[gr.grnet.aquarium.event.model.PolicyEntry]].
    */
   final val POLICY_ENTRIES_COLLECTION = "policyEntries"
 
@@ -423,14 +407,25 @@ object MongoDBStore {
     UserState.fromJson(JSON.serialize(dbObj))
   }
 
-  def dbObjectToWalletEntry(dbObj: DBObject): WalletEntry = {
-    WalletEntry.fromJson(JSON.serialize(dbObj))
+  def firstResultIfExists[A](cursor: DBCursor, f: DBObject ⇒ A): Option[A] = {
+    withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext) {
+        Some(f(cursor.next()))
+      } else {
+        None
+      }
+    }
   }
 
   def dbObjectToPolicyEntry(dbObj: DBObject): PolicyEntry = {
     PolicyEntry.fromJson(JSON.serialize(dbObj))
   }
 
+  def ping(mongo: Mongo): Unit = synchronized {
+    // This requires a network roundtrip
+    mongo.isLocked
+  }
+
   def findBy[A >: Null <: AnyRef](name: String,
                                   value: String,
                                   collection: DBCollection,
@@ -518,7 +513,7 @@ object MongoDBStore {
     obj
   }
 
-  def insertObject[A <: MongoDBEventModel](obj: A, collection: DBCollection, serializer: (A) => DBObject) : ObjectId = {
+  def insertObject[A <: MongoDBEventModel](obj: A, collection: DBCollection, serializer: (A) => DBObject) : Unit = {
     val dbObject = serializer apply obj
     val objectId = obj._id  match {
       case null ⇒
@@ -533,8 +528,6 @@ object MongoDBStore {
     dbObject.put(JsonNames._id, objectId)
 
     collection.insert(dbObject, WriteConcern.JOURNAL_SAFE)
-
-    objectId
   }
 
   def jsonSupportToDBObject(jsonSupport: JsonSupport) = {