WIP Resource event handling
[aquarium] / src / main / scala / gr / grnet / aquarium / store / mongodb / MongoDBStore.scala
index 25981bd..04ecf59 100644 (file)
@@ -36,7 +36,7 @@
 package gr.grnet.aquarium.store.mongodb
 
 import com.mongodb.util.JSON
-import gr.grnet.aquarium.computation.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.model.im.IMEventModel
@@ -44,17 +44,17 @@ 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.model.WalletEntry.{JsonNames ⇒ WalletJsonNames}
 import gr.grnet.aquarium.event.model.PolicyEntry.{JsonNames ⇒ PolicyJsonNames}
-import java.util.Date
 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.UserState
-import gr.grnet.aquarium.event.model.{ExternalEventModel, WalletEntry, PolicyEntry}
+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,7 +102,7 @@ class MongoDBStore(
     MongoDBResourceEvent.fromOther(event, null)
   }
 
-  def pingResourceEventStore(): Unit = {
+  def pingResourceEventStore(): Unit = synchronized {
     MongoDBStore.ping(mongo)
   }
 
@@ -188,9 +186,22 @@ class MongoDBStore(
     MongoDBStore.runQuery[ResourceEvent](query, resourceEvents, orderBy)(MongoDBResourceEvent.fromDBObject)(None)
   }
   
-  def countOutOfSyncEventsForBillingPeriod(userId: String, startMillis: Long, stopMillis: Long): Long = {
-    // 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,
@@ -203,127 +214,38 @@ 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
-  }
-
-  def findLatestUserStateForEndOfBillingMonth(userId: String,
-                                              yearOfBillingMonth: Int,
-                                              billingMonth: Int): Option[UserState] = {
-    None // FIXME: implement
-  }
-
-  def deleteUserState(userId: String) = {
-    val query = new BasicDBObject(UserStateJsonNames.userID, userId)
-    userStates.findAndRemove(query)
-  }
-  //- UserStateStore
-
-  //+WalletEntryStore
-  def storeWalletEntry(entry: WalletEntry): Maybe[RecordID] = {
-    Maybe {
-      MongoDBStore.storeAny[WalletEntry](
-        entry,
-        walletEntries,
-        WalletJsonNames.id,
-        (e) => e.id,
-        MongoDBStore.jsonSupportToDBObject)
-    }
-  }
-
-  def findWalletEntryById(id: String): Maybe[WalletEntry] = {
-    MongoDBStore.findBy(WalletJsonNames.id, id, walletEntries, MongoDBStore.dbObjectToWalletEntry): Maybe[WalletEntry]
-  }
-
-  def findUserWalletEntries(userId: String) = {
-    // TODO: optimize
-    findUserWalletEntriesFromTo(userId, new Date(0), new Date(Int.MaxValue))
-  }
-
-  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 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 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
-
-          while(cursor.hasNext && _ok) {
-            val walletEntry = MongoDBStore.dbObjectToWalletEntry(cursor.next())
-            var currentOccurredMillis = walletEntry.occurredMillis
-            _ok = currentOccurredMillis == _previousOccurredMillis
-            
-            if(_ok) {
-              buffer += walletEntry
-            }
-          }
+    logger.debug("findLatestUserStateForFullMonthBilling(%s, %s) query: %s".format(userID, bmi, query))
 
-          buffer.toList
-        } else {
-          null
-        }
-      } finally {
-        cursor.close()
-      }
-    }
-  }
+    // Descending order, so that the latest comes first
+    val sorter = new BasicDBObject(UserState.JsonNames.occurredMillis, -1)
 
-  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 =>
-    }
+    val cursor = userStates.find(query).sort(sorter)
 
-    MongoDBStore.runQuery[WalletEntry](q, walletEntries)(MongoDBStore.dbObjectToWalletEntry)(Some(_sortByTimestampAsc))
+    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
   }
-  //-WalletEntryStore
+  //- UserStateStore
 
   //+IMEventStore
   def createIMEventFromJson(json: String) = {
@@ -348,18 +270,26 @@ class MongoDBStore(
     MongoDBStore.findBy(IMEventNames.id, id, imEvents, MongoDBIMEvent.fromDBObject)
   }
 
+
+  /**
+   * 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()
+
+    // Normally one such event is allowed ...
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
+
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
+  }
+
   def findLatestIMEventByUserID(userID: String): Option[IMEvent] = {
     val query = new BasicDBObject(IMEventNames.userID, userID)
     val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, -1))
 
-    withCloseable(cursor) { cursor ⇒
-      if(cursor.hasNext) {
-        Some(MongoDBIMEvent.fromDBObject(cursor.next()))
-      } else {
-        None
-      }
-    }
-
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
   }
 
   /**
@@ -440,9 +370,10 @@ object MongoDBStore {
   final val RESOURCE_EVENTS_COLLECTION = "resevents"
 
   /**
-   * Collection holding the snapshots of [[gr.grnet.aquarium.computation.UserState]].
+   * Collection holding the snapshots of [[gr.grnet.aquarium.computation.state.UserState]].
    *
-   * [[gr.grnet.aquarium.computation.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"
 
@@ -463,13 +394,6 @@ object MongoDBStore {
   final val UNPARSED_IM_EVENTS_COLLECTION = "unparsed_imevents"
 
   /**
-   * Collection holding [[gr.grnet.aquarium.event.model.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"
@@ -483,15 +407,21 @@ 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 = {
+  def ping(mongo: Mongo): Unit = synchronized {
     // This requires a network roundtrip
     mongo.isLocked
   }