WIP Resource event handling
[aquarium] / src / main / scala / gr / grnet / aquarium / store / mongodb / MongoDBStore.scala
index 519b77b..04ecf59 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.store.mongodb
 
-import gr.grnet.aquarium.util.Loggable
-import com.ckkloverdos.maybe.{Failed, Just, Maybe}
 import com.mongodb.util.JSON
-import gr.grnet.aquarium.user.UserState
-import gr.grnet.aquarium.user.UserState.JsonNames
-import gr.grnet.aquarium.util.displayableObjectInfo
+import gr.grnet.aquarium.computation.state.UserState.{JsonNames ⇒ UserStateJsonNames}
 import gr.grnet.aquarium.util.json.JsonSupport
-import collection.mutable.{ListBuffer}
+import collection.mutable.ListBuffer
+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.logic.events.{WalletEntry, UserEvent, ResourceEvent, AquariumEvent}
-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 event _store (and soon the user _store).
+ * Mongodb implementation of the various aquarium stores.
  *
  * @author Christos KK Loverdos <loverdos@gmail.com>
  * @author Georgios Gousios <gousiosg@gmail.com>
@@ -59,223 +67,487 @@ class MongoDBStore(
     val database: String,
     val username: String,
     val password: String)
-  extends EventStore with UserStore with WalletStore with Loggable {
+  extends ResourceEventStore
+  with UserStateStore
+  with IMEventStore
+  with PolicyStore
+  with Loggable {
 
-  private[store] lazy val events: DBCollection = getCollection(MongoDBStore.EVENTS_COLLECTION)
-  private[store] lazy val users: DBCollection = getCollection(MongoDBStore.USERS_COLLECTION)
-  private[store] lazy val imevents: DBCollection = getCollection(MongoDBStore.IM_EVENTS_COLLECTION)
-  private[store] lazy val wallets: DBCollection = getCollection(MongoDBStore.IM_WALLETS)
+  override type IMEvent = MongoDBIMEvent
+  override type ResourceEvent = MongoDBResourceEvent
+
+  private[store] lazy val resourceEvents   = getCollection(MongoDBStore.RESOURCE_EVENTS_COLLECTION)
+  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 policyEntries    = getCollection(MongoDBStore.POLICY_ENTRIES_COLLECTION)
 
   private[this] def getCollection(name: String): DBCollection = {
     val db = mongo.getDB(database)
-    if(!db.authenticate(username, password.toCharArray)) {
+    //logger.debug("Authenticating to mongo")
+    if(!db.isAuthenticated && !db.authenticate(username, password.toCharArray)) {
       throw new StoreException("Could not authenticate user %s".format(username))
     }
     db.getCollection(name)
   }
 
-  /* TODO: Some of the following methods rely on JSON (de-)serialization).
-  * A method based on proper object serialization would be much faster.
-  */
-
-  private[this] def _deserializeEvent[A <: AquariumEvent](a: DBObject): A = {
-    //TODO: Distinguish events and deserialize appropriately
-    ResourceEvent.fromJson(JSON.serialize(a)).asInstanceOf[A]
+  private[this] def _sortByTimestampAsc[A <: ExternalEventModel](one: A, two: A): Boolean = {
+    if (one.occurredMillis > two.occurredMillis) false
+    else if (one.occurredMillis < two.occurredMillis) true
+    else true
   }
 
-  private[this] def _deserializeUserState(dbObj: DBObject): UserState = {
-    val jsonString = JSON.serialize(dbObj)
-    UserState.fromJson(jsonString)
+  //+ResourceEventStore
+  def createResourceEventFromOther(event: ResourceEventModel): ResourceEvent = {
+    MongoDBResourceEvent.fromOther(event, null)
   }
 
-  private[this] def _makeDBObject(any: JsonSupport): DBObject = {
-    JSON.parse(any.toJson) match {
-      case dbObject: DBObject ⇒
-        dbObject
-      case _ ⇒
-        throw new StoreException("Could not transform %s -> %s".format(displayableObjectInfo(any), classOf[DBObject].getName))
-    }
+  def pingResourceEventStore(): Unit = synchronized {
+    MongoDBStore.ping(mongo)
   }
 
-  private[this] def _prepareFieldQuery(name: String, value: String): DBObject = {
-    val dbObj = new BasicDBObject(1)
-    dbObj.put(name, value)
-    dbObj
+  def insertResourceEvent(event: ResourceEventModel) = {
+    val localEvent = MongoDBResourceEvent.fromOther(event, new ObjectId().toStringMongod)
+    MongoDBStore.insertObject(localEvent, resourceEvents, MongoDBStore.jsonSupportToDBObject)
+    localEvent
   }
 
-  private[this] def _insertObject(collection: DBCollection, obj: JsonSupport): DBObject = {
-    val dbObj = _makeDBObject(obj)
-    collection insert dbObj
-    dbObj
+  def findResourceEventById(id: String): Option[ResourceEvent] = {
+    MongoDBStore.findBy(ResourceEventNames.id, id, resourceEvents, MongoDBResourceEvent.fromDBObject)
   }
 
-  private[this] def _checkWasInserted(collection: DBCollection, obj: JsonSupport,  idName: String, id: String): String = {
-    val cursor = collection.find(_prepareFieldQuery(idName, id))
-    if (!cursor.hasNext) {
-      val errMsg = "Failed to _store %s".format(displayableObjectInfo(obj))
-      logger.error(errMsg)
-      throw new StoreException(errMsg)
-    }
+  def findResourceEventsByUserId(userId: String)
+                                (sortWith: Option[(ResourceEvent, ResourceEvent) => Boolean]): List[ResourceEvent] = {
+    val query = new BasicDBObject(ResourceEventNames.userID, userId)
 
-    val retval = cursor.next.get("_id").toString
-    cursor.close()
-    retval
+    MongoDBStore.runQuery(query, resourceEvents)(MongoDBResourceEvent.fromDBObject)(sortWith)
   }
 
-  private[this] def _store[A <: AquariumEvent](entry: A, col: DBCollection) : Maybe[RecordID] = {
-    try {
-      // Store
-      val dbObj = _makeDBObject(entry)
-      col.insert(dbObj)
+  def findResourceEventsByUserIdAfterTimestamp(userId: String, timestamp: Long): List[ResourceEvent] = {
+    val query = new BasicDBObject()
+    query.put(ResourceEventNames.userID, userId)
+    query.put(ResourceEventNames.occurredMillis, new BasicDBObject("$gt", timestamp))
+    
+    val sort = new BasicDBObject(ResourceEventNames.occurredMillis, 1)
 
-      // Get back to retrieve unique id
-      val cursor = col.find(_prepareFieldQuery("id", entry.id))
+    val cursor = resourceEvents.find(query).sort(sort)
 
-      if (!cursor.hasNext) {
-        cursor.close()
-        logger.error("Failed to _store entry: %s".format(entry))
-        return Failed(new StoreException("Failed to _store entry: %s".format(entry)))
+    try {
+      val buffer = new scala.collection.mutable.ListBuffer[ResourceEvent]
+      while(cursor.hasNext) {
+        buffer += MongoDBResourceEvent.fromDBObject(cursor.next())
       }
+      buffer.toList.sortWith(_sortByTimestampAsc)
+    } finally {
+      cursor.close()
+    }
+  }
+
+  def findResourceEventHistory(userId: String, resName: String,
+                               instid: Option[String], upTo: Long) : List[ResourceEvent] = {
+    val query = new BasicDBObject()
+    query.put(ResourceEventNames.userID, userId)
+    query.put(ResourceEventNames.occurredMillis, new BasicDBObject("$lt", upTo))
+    query.put(ResourceEventNames.resource, resName)
+
+    instid match {
+      case Some(id) =>
+        Policy.policy.findResource(resName) match {
+          case Some(y) => query.put(ResourceEventNames.details,
+            new BasicDBObject(y.descriminatorField, instid.get))
+          case None =>
+        }
+      case None =>
+    }
 
-      val retval = Just(RecordID(cursor.next.get("_id").toString))
+    val sort = new BasicDBObject(ResourceEventNames.occurredMillis, 1)
+    val cursor = resourceEvents.find(query).sort(sort)
+
+    try {
+      val buffer = new scala.collection.mutable.ListBuffer[ResourceEvent]
+      while(cursor.hasNext) {
+        buffer += MongoDBResourceEvent.fromDBObject(cursor.next())
+      }
+      buffer.toList.sortWith(_sortByTimestampAsc)
+    } finally {
       cursor.close()
-      retval
-    } catch {
-      case m: MongoException =>
-        logger.error("Unknown Mongo error: %s".format(m)); Failed(m)
     }
   }
 
-  private[this] def _findById[A <: AquariumEvent](id: String, col: DBCollection) : Option[A] = {
-    val q = new BasicDBObject()
-    q.put("id", id)
+  def findResourceEventsForReceivedPeriod(userId: String, startTimeMillis: Long, stopTimeMillis: Long): List[ResourceEvent] = {
+    val query = new BasicDBObject()
+    query.put(ResourceEventNames.userID, userId)
+    query.put(ResourceEventNames.receivedMillis, new BasicDBObject("$gte", startTimeMillis))
+    query.put(ResourceEventNames.receivedMillis, new BasicDBObject("$lte", stopTimeMillis))
 
-    val cur = col.find(q)
+    // Sort them by increasing order for occurred time
+    val orderBy = new BasicDBObject(ResourceEventNames.occurredMillis, 1)
 
-    val retval = if (cur.hasNext)
-      Some(_deserializeEvent(cur.next))
-    else
-      None
-    
-    cur.close()
-    retval
+    MongoDBStore.runQuery[ResourceEvent](query, resourceEvents, orderBy)(MongoDBResourceEvent.fromDBObject)(None)
   }
   
-  private[this] def _query[A <: AquariumEvent](q: BasicDBObject,
-                                              col: DBCollection)
-                                              (sortWith: Option[(A, A) => Boolean]): List[A] = {
-    val cur = col.find(q)
-    if (!cur.hasNext) {
-      cur.close()
-      return List()
-    }
+  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)
+  }
 
-    val buff = new ListBuffer[A]()
+  def findAllRelevantResourceEventsForBillingPeriod(userId: String,
+                                                    startMillis: Long,
+                                                    stopMillis: Long): List[ResourceEvent] = {
+    // FIXME: Implement
+    Nil
+  }
+  //-ResourceEventStore
+
+  //+ UserStateStore
+  def insertUserState(userState: UserState) = {
+    MongoDBStore.insertUserState(
+      userState.copy(_id = new ObjectId().toString),
+      userStates,
+      MongoDBStore.jsonSupportToDBObject
+    )
+  }
 
-    while(cur.hasNext)
-      buff += _deserializeEvent(cur.next)
+  def findUserStateByUserID(userID: String): Option[UserState] = {
+    val query = new BasicDBObject(UserStateJsonNames.userID, userID)
+    val cursor = userStates find query
 
-    cur.close()
-    
-    sortWith match {
-      case Some(sorter) => buff.toList.sortWith(sorter)
-      case None => buff.toList
-    }
+    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
   }
 
-  private[this] def _sortByTimestampAsc[A <: AquariumEvent](one: A, two: A): Boolean = {
-    if (one.timestamp > two.timestamp) false
-    else if (one.timestamp < two.timestamp) true
-    else true
+  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()
+
+    logger.debug("findLatestUserStateForFullMonthBilling(%s, %s) query: %s".format(userID, bmi, query))
+
+    // Descending order, so that the latest comes first
+    val sorter = new BasicDBObject(UserState.JsonNames.occurredMillis, -1)
+
+    val cursor = userStates.find(query).sort(sorter)
+
+    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
   }
+  //- UserStateStore
 
-  private[this] def _sortByTimestampDesc[A <: AquariumEvent](one: A, two: A): Boolean = {
-    if (one.timestamp < two.timestamp) false
-    else if (one.timestamp > two.timestamp) true
-    else true
+  //+IMEventStore
+  def createIMEventFromJson(json: String) = {
+    MongoDBStore.createIMEventFromJson(json)
   }
 
-  //+EventStore
-  def storeEvent[A <: AquariumEvent](event: A): Maybe[RecordID] = _store(event, events)
+  def createIMEventFromOther(event: IMEventModel) = {
+    MongoDBStore.createIMEventFromOther(event)
+  }
 
-  def findEventById[A <: AquariumEvent](id: String): Option[A] = _findById[A](id, events)
+  def pingIMEventStore(): Unit = {
+    MongoDBStore.ping(mongo)
+  }
 
-  def findEventsByUserId[A <: AquariumEvent](userId: String)
-                                            (sortWith: Option[(A, A) => Boolean]): List[A] = {
-    val q = new BasicDBObject()
-    q.put("userId", userId)
+  def insertIMEvent(event: IMEventModel): IMEvent = {
+    val localEvent = MongoDBIMEvent.fromOther(event, new ObjectId().toStringMongod)
+    MongoDBStore.insertObject(localEvent, imEvents, MongoDBStore.jsonSupportToDBObject)
+    localEvent
+  }
 
-    _query(q, events)(sortWith)
+  def findIMEventById(id: String): Option[IMEvent] = {
+    MongoDBStore.findBy(IMEventNames.id, id, imEvents, MongoDBIMEvent.fromDBObject)
   }
 
-  def findEventsByUserIdAfterTimestamp[A <: AquariumEvent](userId: String, timestamp: Long): List[A] = {
-    val query = new BasicDBObject()
-    query.put("userId", userId)
-    query.put("timestamp", "{\"$gte\": %s}".format(timestamp))
-    
-    val sort = new BasicDBObject()
-    sort.put("timestamp", 1)
 
-    val cursor = events.find(query).sort(sort)
-    val buffer = new scala.collection.mutable.ListBuffer[A]
-    while(cursor.hasNext) {
-      buffer += _deserializeEvent(cursor.next())
-    }
+  /**
+   * 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()
 
-    cursor.close()
+    // Normally one such event is allowed ...
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
 
-    buffer.toList
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
   }
-  //-EventStore
 
-  //+UserStore
+  def findLatestIMEventByUserID(userID: String): Option[IMEvent] = {
+    val query = new BasicDBObject(IMEventNames.userID, userID)
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, -1))
 
-  def storeUserState(userState: UserState): Maybe[RecordID] = {
-    Maybe {
-      val dbObj = _insertObject(users, userState)
-      val id    = _checkWasInserted(users, userState, JsonNames.userId, userState.userId)
-      RecordID(id)
-    }
+    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
   }
 
-  def findUserStateByUserId(userId: String): Maybe[UserState] = {
-    Maybe {
-      val queryObj = _prepareFieldQuery(JsonNames.userId, userId)
-      val cursor = events.find(queryObj)
+  /**
+   * 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()
 
-      if(!cursor.hasNext) {
-        cursor.close()
-        null
+    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
+
+    withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext) {
+        Some(MongoDBIMEvent.fromDBObject(cursor.next()))
       } else {
-        val userState = _deserializeUserState(cursor.next())
-        cursor.close()
-        userState
+        None
+      }
+   }
+  }
+
+  /**
+   * 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)
       }
     }
   }
-  //-UserStore
+  //-IMEventStore
 
-  //+WalletStore
-  def store(entry: WalletEntry): Maybe[RecordID] = _store(entry, wallets)
 
-  def findEntryById(id: String): Option[WalletEntry] = _findById[WalletEntry](id, wallets)
 
-  def findAllUserEntries(userId: String) = findUserEntriesFromTo(userId, new Date(0), new Date(Int.MaxValue))
+  //+PolicyStore
+  def loadPolicyEntriesAfter(after: Long): List[PolicyEntry] = {
+    val query = new BasicDBObject(PolicyEntry.JsonNames.validFrom,
+      new BasicDBObject("$gt", after))
+    MongoDBStore.runQuery(query, policyEntries)(MongoDBStore.dbObjectToPolicyEntry)(Some(_sortByTimestampAsc))
+  }
+
+  def storePolicyEntry(policy: PolicyEntry): Maybe[RecordID] = MongoDBStore.storePolicyEntry(policy, policyEntries)
 
-  def findUserEntriesFromTo(userId: String, from: Date, to: Date) : List[WalletEntry] = {
-    val q = new BasicDBObject()
-    q.put("timestamp", new BasicDBObject("$gt", from.getTime))
-    q.put("timestamp", new BasicDBObject("$lt", to.getTime))
-    q.put("userId", userId)
 
-    _query[WalletEntry](q, wallets)(Some(_sortByTimestampAsc))
+  def updatePolicyEntry(policy: PolicyEntry) = {
+    //Find the entry
+    val query = new BasicDBObject(PolicyEntry.JsonNames.id, policy.id)
+    val policyObject = MongoDBStore.jsonSupportToDBObject(policy)
+    policyEntries.update(query, policyObject, true, false)
   }
-  //-WalletStore
+  
+  def findPolicyEntry(id: String) = {
+    MongoDBStore.findBy(PolicyJsonNames.id, id, policyEntries, MongoDBStore.dbObjectToPolicyEntry)
+  }
+
+  //-PolicyStore
 }
 
 object MongoDBStore {
-  def EVENTS_COLLECTION = "events"
-  def USERS_COLLECTION = "users"
-  def IM_EVENTS_COLLECTION = "imevents"
-  def IM_WALLETS = "wallets"
+  object JsonNames {
+    final val _id = "_id"
+  }
+
+  /**
+   * 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.computation.state.UserState]].
+   *
+   * [[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.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.model.im.IMEventModel]]s that could not be parsed to normal objects.
+   *
+   * We of course assume at least a valid JSON representation.
+   *
+   * User events are coming from the IM module (external).
+   */
+  final val UNPARSED_IM_EVENTS_COLLECTION = "unparsed_imevents"
+
+  /**
+   * Collection holding [[gr.grnet.aquarium.logic.accounting.dsl.DSLPolicy]].
+   */
+//  final val POLICIES_COLLECTION = "policies"
+
+  /**
+   * Collection holding [[gr.grnet.aquarium.event.model.PolicyEntry]].
+   */
+  final val POLICY_ENTRIES_COLLECTION = "policyEntries"
+
+  def dbObjectToUserState(dbObj: DBObject): UserState = {
+    UserState.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,
+                                  deserializer: (DBObject) => A) : Option[A] = {
+    val query = new BasicDBObject(name, value)
+    val cursor = collection find query
+
+    withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext)
+        Some(deserializer apply cursor.next)
+      else
+        None
+    }
+  }
+
+  def runQuery[A <: ExternalEventModel](query: DBObject, collection: DBCollection, orderBy: DBObject = null)
+                                  (deserializer: (DBObject) => A)
+                                  (sortWith: Option[(A, A) => Boolean]): List[A] = {
+    val cursor0 = collection find query
+    val cursor = if(orderBy ne null) {
+      cursor0 sort orderBy
+    } else {
+      cursor0
+    } // I really know that docs say that it is the same cursor.
+
+    if(!cursor.hasNext) {
+      cursor.close()
+      Nil
+    } else {
+      val buff = new ListBuffer[A]()
+
+      while(cursor.hasNext) {
+        buff += deserializer apply cursor.next
+      }
+
+      cursor.close()
+
+      sortWith match {
+        case Some(sorter) => buff.toList.sortWith(sorter)
+        case None => buff.toList
+      }
+    }
+  }
+
+  def storeUserState(userState: UserState, collection: DBCollection) = {
+    storeAny[UserState](userState, collection, ResourceEventNames.userID, _.userID, MongoDBStore.jsonSupportToDBObject)
+  }
+  
+  def storePolicyEntry(policyEntry: PolicyEntry, collection: DBCollection): Maybe[RecordID] = {
+    Maybe(storeAny[PolicyEntry](policyEntry, collection, PolicyJsonNames.id, _.id, MongoDBStore.jsonSupportToDBObject))
+  }
+
+  def storeAny[A](any: A,
+                  collection: DBCollection,
+                  idName: String,
+                  idValueProvider: (A) => String,
+                  serializer: (A) => DBObject) : RecordID = {
+
+    val dbObject = serializer apply any
+    val _id = new ObjectId()
+    dbObject.put("_id", _id)
+    val writeResult = collection.insert(dbObject, WriteConcern.JOURNAL_SAFE)
+    writeResult.getLastError().throwOnError()
+
+    RecordID(dbObject.get("_id").toString)
+  }
+
+  // FIXME: consolidate
+  def insertUserState[A <: UserState](obj: A, collection: DBCollection, serializer: A ⇒ DBObject) = {
+    val dbObject = serializer apply obj
+    val objectId = obj._id  match {
+      case null ⇒
+        val _id = new ObjectId()
+        dbObject.put("_id", _id)
+        _id
+
+      case _id ⇒
+        _id
+    }
+
+    dbObject.put(JsonNames._id, objectId)
+
+    collection.insert(dbObject, WriteConcern.JOURNAL_SAFE)
+
+    obj
+  }
+
+  def insertObject[A <: MongoDBEventModel](obj: A, collection: DBCollection, serializer: (A) => DBObject) : Unit = {
+    val dbObject = serializer apply obj
+    val objectId = obj._id  match {
+      case null ⇒
+        val _id = new ObjectId()
+        dbObject.put("_id", _id)
+        _id
+
+      case _id ⇒
+        _id
+    }
+
+    dbObject.put(JsonNames._id, objectId)
+
+    collection.insert(dbObject, WriteConcern.JOURNAL_SAFE)
+  }
+
+  def jsonSupportToDBObject(jsonSupport: JsonSupport) = {
+    Conversions.jsonSupportToDBObject(jsonSupport)
+  }
+
+  final def isLocalIMEvent(event: IMEventModel) = event match {
+    case _: MongoDBIMEvent ⇒ true
+    case _ ⇒ false
+  }
+
+  final def createIMEventFromJson(json: String) = {
+    MongoDBIMEvent.fromJsonString(json)
+  }
+
+  final def createIMEventFromOther(event: IMEventModel) = {
+    MongoDBIMEvent.fromOther(event, null)
+  }
+
+  final def createIMEventFromJsonBytes(jsonBytes: Array[Byte]) = {
+    MongoDBIMEvent.fromJsonBytes(jsonBytes)
+  }
 }