Merge branch 'snapshots'
[aquarium] / src / main / scala / gr / grnet / aquarium / store / mongodb / MongoDBStore.scala
index 48b5a2d..1433e48 100644 (file)
 
 package gr.grnet.aquarium.store.mongodb
 
-import com.mongodb.util.JSON
-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
-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 collection.immutable
 import com.mongodb._
-import org.bson.types.ObjectId
-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
 import gr.grnet.aquarium.computation.BillingMonthInfo
-import gr.grnet.aquarium.policy.PolicyModel
-import gr.grnet.aquarium.{Aquarium, AquariumException}
-import collection.immutable.SortedMap
+import gr.grnet.aquarium.converter.StdConverters
 import gr.grnet.aquarium.logic.accounting.dsl.Timeslot
+import gr.grnet.aquarium.message.MessageConstants
+import gr.grnet.aquarium.message.avro.gen.{UserAgreementHistoryMsg, UserStateMsg, IMEventMsg, ResourceEventMsg, PolicyMsg}
+import gr.grnet.aquarium.message.avro.{MessageHelpers, MessageFactory, OrderingHelpers, AvroHelpers}
+import gr.grnet.aquarium.store._
+import gr.grnet.aquarium.util._
+import gr.grnet.aquarium.util.Once
+import gr.grnet.aquarium.util.json.JsonSupport
+import gr.grnet.aquarium.{Aquarium, AquariumException}
+import org.apache.avro.specific.SpecificRecord
+import org.bson.types.ObjectId
 
 /**
  * Mongodb implementation of the various aquarium stores.
@@ -75,61 +70,97 @@ class MongoDBStore(
   with PolicyStore
   with Loggable {
 
-  override type IMEvent = MongoDBIMEvent
-  override type ResourceEvent = MongoDBResourceEvent
-  override type Policy = MongoDBPolicy
-
-  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 policies = getCollection(MongoDBStore.POLICY_COLLECTION)
+  private[store] lazy val resourceEvents = getCollection(MongoDBStore.ResourceEventCollection)
+  private[store] lazy val userStates = getCollection(MongoDBStore.UserStateCollection)
+  private[store] lazy val imEvents = getCollection(MongoDBStore.IMEventCollection)
+  private[store] lazy val policies = getCollection(MongoDBStore.PolicyCollection)
+
+  private[store] lazy val indicesMap = {
+   val resev=  new BasicDBObjectBuilder().
+                      add(MongoDBStore.JsonNames.id,1).
+                      add(MongoDBStore.JsonNames.userID,1).
+                      add(MongoDBStore.JsonNames.occurredMillis,1).
+                      add(MongoDBStore.JsonNames.receivedMillis,1).get
+   val imev =  new BasicDBObjectBuilder().
+                 add(MongoDBStore.JsonNames.userID,1).
+                 add(MongoDBStore.JsonNames.eventType,"").
+                 add(MongoDBStore.JsonNames.occurredMillis,1).get
+   val policy = new BasicDBObjectBuilder().
+                 add("validFromMillis",1).
+                 add("validToMillis",1).get
+   val user = new BasicDBObjectBuilder().
+              add( "occurredMillis",1).
+              add("isFullBillingMonth",false).
+              add("billingYear",1).
+              add("billingMonth",1).
+              add("billingMonthDay",1).get
+    Map(MongoDBStore.ResourceEventCollection -> resev,
+        MongoDBStore.IMEventCollection-> imev,
+        MongoDBStore.PolicyCollection-> policy,
+        MongoDBStore.UserStateCollection-> user
+       )
+  }
+  private[this] val once = new Once()
+
+  private[this] def doAuthenticate(db: DB) {
+    if(!db.isAuthenticated && !db.authenticate(username, password.toCharArray)) {
+      throw new AquariumException("Could not authenticate user %s".format(username))
+    }
+  }
 
   private[this] def getCollection(name: String): DBCollection = {
     val db = mongo.getDB(database)
-    //logger.debug("Authenticating to mongo")
-    if(!db.isAuthenticated && !db.authenticate(username, password.toCharArray)) {
-      throw new AquariumException("Could not authenticate user %s".format(username))
+    doAuthenticate(db)
+    once.run { /* this code is thread-safe and will run exactly once*/
+      indicesMap.foreach { case (collection,obj) =>
+        mongo.getDB(database).getCollection(collection).createIndex(obj)
+      }
     }
     db.getCollection(name)
   }
 
   //+ResourceEventStore
-  def createResourceEventFromOther(event: ResourceEventModel): ResourceEvent = {
-    MongoDBResourceEvent.fromOther(event, null)
-  }
-
   def pingResourceEventStore(): Unit = synchronized {
+    getCollection(MongoDBStore.ResourceEventCollection)
     MongoDBStore.ping(mongo)
   }
 
-  def insertResourceEvent(event: ResourceEventModel) = {
-    val localEvent = MongoDBResourceEvent.fromOther(event, new ObjectId().toStringMongod)
-    MongoDBStore.insertObject(localEvent, resourceEvents, MongoDBStore.jsonSupportToDBObject)
-    localEvent
-  }
+  def insertResourceEvent(event: ResourceEventMsg) = {
+    val mongoID = new ObjectId()
+    event.setInStoreID(mongoID.toStringMongod)
 
-  def findResourceEventByID(id: String): Option[ResourceEvent] = {
-    MongoDBStore.findBy(ResourceEventNames.id, id, resourceEvents, MongoDBResourceEvent.fromDBObject)
-  }
+    val dbObject = new BasicDBObjectBuilder().
+      add(MongoDBStore.JsonNames._id, mongoID).
+      add(MongoDBStore.JsonNames.payload, AvroHelpers.bytesOfSpecificRecord(event)).
+      add(MongoDBStore.JsonNames.userID, event.getUserID).
+      add(MongoDBStore.JsonNames.occurredMillis, event.getOccurredMillis).
+      add(MongoDBStore.JsonNames.receivedMillis, event.getReceivedMillis).
+    get()
 
-  def findResourceEventsByUserID(userId: String)
-                                (sortWith: Option[(ResourceEvent, ResourceEvent) => Boolean]): List[ResourceEvent] = {
-    val query = new BasicDBObject(ResourceEventNames.userID, userId)
+    MongoDBStore.insertDBObject(dbObject, resourceEvents)
+    event
+  }
 
-    MongoDBStore.runQuery(query, resourceEvents)(MongoDBResourceEvent.fromDBObject)(sortWith)
+  def findResourceEventByID(id: String): Option[ResourceEventMsg] = {
+    val dbObjectOpt = MongoDBStore.findOneByAttribute(resourceEvents, MongoDBStore.JsonNames.id, id)
+    for {
+      dbObject ← dbObjectOpt
+      payload = dbObject.get(MongoDBStore.JsonNames.payload)
+      msg = AvroHelpers.specificRecordOfBytes(payload.asInstanceOf[Array[Byte]], new ResourceEventMsg)
+    } yield msg
   }
 
   def countOutOfSyncResourceEventsForBillingPeriod(userID: String, startMillis: Long, stopMillis: Long): Long = {
     val query = new BasicDBObjectBuilder().
-      add(ResourceEventModel.Names.userID, userID).
+      add(MongoDBStore.JsonNames.userID, userID).
       // received within the period
-      add(ResourceEventModel.Names.receivedMillis, new BasicDBObject("$gte", startMillis)).
-      add(ResourceEventModel.Names.receivedMillis, new BasicDBObject("$lte", stopMillis)).
+      add(MongoDBStore.JsonNames.receivedMillis, new BasicDBObject("$gte", startMillis)).
+      add(MongoDBStore.JsonNames.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.add(0, new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, new BasicDBObject("$lt", startMillis)))
+        dbList.add(1, new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, new BasicDBObject("$gt", stopMillis)))
         dbList
       }).
       get()
@@ -141,104 +172,161 @@ class MongoDBStore(
       userID: String,
       startMillis: Long,
       stopMillis: Long
-  )(f: ResourceEvent ⇒ Unit): Unit = {
-
+  )(f: ResourceEventMsg ⇒ Unit): Long = {
+    var _counter= 0L
     val query = new BasicDBObjectBuilder().
-      add(ResourceEventModel.Names.userID, userID).
-      add(ResourceEventModel.Names.occurredMillis, new BasicDBObject("$gte", startMillis)).
-      add(ResourceEventModel.Names.occurredMillis, new BasicDBObject("$lte", stopMillis)).
+      add(MongoDBStore.JsonNames.userID, userID).
+      add(MongoDBStore.JsonNames.occurredMillis, new BasicDBObject("$gte", startMillis)).
+      add(MongoDBStore.JsonNames.occurredMillis, new BasicDBObject("$lte", stopMillis)).
       get()
 
-    val sorter = new BasicDBObject(ResourceEventModel.Names.occurredMillis, 1)
+    val sorter = new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, 1)
     val cursor = resourceEvents.find(query).sort(sorter)
 
     withCloseable(cursor) { cursor ⇒
       while(cursor.hasNext) {
         val nextDBObject = cursor.next()
-        val nextEvent = MongoDBResourceEvent.fromDBObject(nextDBObject)
+        val payload = nextDBObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+        val nextEvent = AvroHelpers.specificRecordOfBytes(payload, new ResourceEventMsg)
 
         f(nextEvent)
+        _counter += 1
       }
     }
+
+    _counter
   }
   //-ResourceEventStore
 
   //+ UserStateStore
-  def insertUserState(userState: UserState) = {
-    MongoDBStore.insertObject(
-      userState.copy(_id = new ObjectId().toString),
-      userStates,
-      MongoDBStore.jsonSupportToDBObject
-    )
+  def findUserStateByUserID(userID: String) = {
+    val dbObjectOpt = MongoDBStore.findOneByAttribute(userStates, MongoDBStore.JsonNames.userID, userID)
+    for {
+      dbObject <- dbObjectOpt
+      payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+      msg = AvroHelpers.specificRecordOfBytes(payload, new UserStateMsg)
+    } yield {
+      msg
+    }
   }
 
-  def findUserStateByUserID(userID: String): Option[UserState] = {
-    val query = new BasicDBObject(UserStateJsonNames.userID, userID)
-    val cursor = userStates find query
+  def findLatestUserStateForFullMonthBilling(userID: String, bmi: BillingMonthInfo) = {
+    val query = new BasicDBObjectBuilder().
+      add(MongoDBStore.JsonNames.userID, userID).
+      add(MongoDBStore.JsonNames.isForFullMonth, true).
+      add(MongoDBStore.JsonNames.billingYear, bmi.year).
+      add(MongoDBStore.JsonNames.billingMonth, bmi.month).
+      get()
+
+    // Descending order, so that the latest comes first
+    val sorter = new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, -1)
 
-    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
+    val cursor = userStates.find(query).sort(sorter)
+
+    withCloseable(cursor) { cursor ⇒
+      MongoDBStore.findNextPayloadRecord(cursor, new UserStateMsg)
+    }
   }
 
-  def findLatestUserStateForFullMonthBilling(userID: String, bmi: BillingMonthInfo): Option[UserState] = {
+  def findLatestUserState(userID: String) = {
     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).
+      add(MongoDBStore.JsonNames.userID, userID).
       get()
 
     // Descending order, so that the latest comes first
-    val sorter = new BasicDBObject(UserState.JsonNames.occurredMillis, -1)
+    val sorter = new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, -1)
 
     val cursor = userStates.find(query).sort(sorter)
 
-    MongoDBStore.firstResultIfExists(cursor, MongoDBStore.dbObjectToUserState)
-  }
-  //- UserStateStore
-
-  //+IMEventStore
-  def createIMEventFromJson(json: String) = {
-    MongoDBStore.createIMEventFromJson(json)
+    withCloseable(cursor) { cursor ⇒
+      MongoDBStore.findNextPayloadRecord(cursor, new UserStateMsg)
+    }
   }
 
-  def createIMEventFromOther(event: IMEventModel) = {
-    MongoDBStore.createIMEventFromOther(event)
+  /**
+   * Stores a user state.
+   */
+  def insertUserState(event: UserStateMsg)= {
+    val mongoID = new ObjectId()
+    event.setInStoreID(mongoID.toStringMongod)
+
+    val dbObject = new BasicDBObjectBuilder().
+      add(MongoDBStore.JsonNames._id, mongoID).
+      add(MongoDBStore.JsonNames.payload, AvroHelpers.bytesOfSpecificRecord(event)).
+      add(MongoDBStore.JsonNames.userID, event.getUserID).
+      add(MongoDBStore.JsonNames.occurredMillis, event.getOccurredMillis).
+      add(MongoDBStore.JsonNames.isForFullMonth, event.getIsForFullMonth).
+      add(MongoDBStore.JsonNames.billingYear, event.getBillingYear).
+      add(MongoDBStore.JsonNames.billingMonth, event.getBillingMonth).
+      add(MongoDBStore.JsonNames.billingMonthDay, event.getBillingMonthDay).
+    get()
+
+    MongoDBStore.insertDBObject(dbObject, userStates)
+    event
   }
+  //- UserStateStore
 
+  //+IMEventStore
   def pingIMEventStore(): Unit = {
+    getCollection(MongoDBStore.IMEventCollection)
     MongoDBStore.ping(mongo)
   }
 
-  def insertIMEvent(event: IMEventModel): IMEvent = {
-    val localEvent = MongoDBIMEvent.fromOther(event, new ObjectId().toStringMongod)
-    MongoDBStore.insertObject(localEvent, imEvents, MongoDBStore.jsonSupportToDBObject)
-    localEvent
+  def insertIMEvent(event: IMEventMsg) = {
+    val mongoID = new ObjectId()
+    event.setInStoreID(mongoID.toStringMongod)
+
+    val dbObject = new BasicDBObjectBuilder().
+      add(MongoDBStore.JsonNames._id, mongoID).
+      add(MongoDBStore.JsonNames.payload, AvroHelpers.bytesOfSpecificRecord(event)).
+      add(MongoDBStore.JsonNames.userID, event.getUserID).
+      add(MongoDBStore.JsonNames.eventType, event.getEventType().toLowerCase).
+      add(MongoDBStore.JsonNames.occurredMillis, event.getOccurredMillis).
+      add(MongoDBStore.JsonNames.receivedMillis, event.getReceivedMillis).
+    get()
+
+    MongoDBStore.insertDBObject(dbObject, imEvents)
+    event
   }
 
-  def findIMEventByID(id: String): Option[IMEvent] = {
-    MongoDBStore.findBy(IMEventNames.id, id, imEvents, MongoDBIMEvent.fromDBObject)
+  def findIMEventByID(id: String) = {
+    val dbObjectOpt = MongoDBStore.findOneByAttribute(imEvents, MongoDBStore.JsonNames.id, id)
+    for {
+      dbObject ← dbObjectOpt
+      payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+      msg = AvroHelpers.specificRecordOfBytes(payload, new IMEventMsg)
+    } yield {
+      msg
+    }
   }
 
 
   /**
    * Find the `CREATE` even for the given user. Note that there must be only one such event.
    */
-  def findCreateIMEventByUserID(userID: String): Option[IMEvent] = {
+  def findCreateIMEventByUserID(userID: String) = {
     val query = new BasicDBObjectBuilder().
-      add(IMEventNames.userID, userID).
-      add(IMEventNames.eventType, IMEventModel.EventTypeNames.create).get()
+      add(MongoDBStore.JsonNames.userID, userID).
+      add(MongoDBStore.JsonNames.eventType, MessageConstants.IMEventMsg.EventTypes.create).get()
 
     // Normally one such event is allowed ...
-    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
+    val cursor = imEvents.find(query).sort(new BasicDBObject(MongoDBStore.JsonNames.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))
+    val dbObjectOpt = withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext) {
+        Some(cursor.next())
+      } else {
+        None
+      }
+    }
 
-    MongoDBStore.firstResultIfExists(cursor, MongoDBIMEvent.fromDBObject)
+    for {
+      dbObject <- dbObjectOpt
+      payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+      msg = AvroHelpers.specificRecordOfBytes(payload, new IMEventMsg)
+    } yield {
+      msg
+    }
   }
 
   /**
@@ -247,86 +335,85 @@ class MongoDBStore(
    *
    * Any exception is propagated to the caller. The underlying DB resources are properly disposed in any case.
    */
-  def foreachIMEventInOccurrenceOrder(userID: String)(f: (IMEvent) => Unit) = {
-    val query = new BasicDBObject(IMEventNames.userID, userID)
-    val cursor = imEvents.find(query).sort(new BasicDBObject(IMEventNames.occurredMillis, 1))
+  def foreachIMEventInOccurrenceOrder(userID: String)(f: (IMEventMsg) ⇒ Boolean) = {
+    val query = new BasicDBObject(MongoDBStore.JsonNames.userID, userID)
+    val cursor = imEvents.find(query).sort(new BasicDBObject(MongoDBStore.JsonNames.occurredMillis, 1))
 
+    var _shouldContinue = true
     withCloseable(cursor) { cursor ⇒
-      while(cursor.hasNext) {
-        val model = MongoDBIMEvent.fromDBObject(cursor.next())
-        f(model)
+      while(_shouldContinue && cursor.hasNext) {
+        val dbObject = cursor.next()
+        val payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+        val msg = AvroHelpers.specificRecordOfBytes(payload, new IMEventMsg)
+
+        _shouldContinue = f(msg)
       }
     }
+
+    _shouldContinue
   }
   //-IMEventStore
 
-
-
   //+PolicyStore
-  def loadPoliciesAfter(after: Long): List[Policy] = {
-    // FIXME implement
-    throw new UnsupportedOperationException
+  def foreachPolicy[U](f: PolicyMsg ⇒ U) {
+    val cursor = policies.find()
+    withCloseable(cursor) { cursor ⇒
+      while(cursor.hasNext) {
+        val dbObject = cursor.next()
+        val payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+        val policy = AvroHelpers.specificRecordOfBytes(payload, new PolicyMsg)
+        f(policy)
+      }
+    }
   }
 
+  def insertPolicy(policy: PolicyMsg): PolicyMsg = {
+    val mongoID = new ObjectId()
+    policy.setInStoreID(mongoID.toStringMongod)
+    val dbObject = new BasicDBObjectBuilder().
+      add(MongoDBStore.JsonNames._id, mongoID).
+      add(MongoDBStore.JsonNames.validFromMillis, policy.getValidFromMillis).
+      add(MongoDBStore.JsonNames.validToMillis, policy.getValidToMillis).
+      add(MongoDBStore.JsonNames.payload, AvroHelpers.bytesOfSpecificRecord(policy)).
+    get()
 
-  def findPolicyByID(id: String) = {
-    // FIXME implement
-    throw new UnsupportedOperationException
+    MongoDBStore.insertDBObject(dbObject, policies)
+    policy
   }
 
-  /**
-   * Store an accounting policy.
-   */
-  def insertPolicy(policy: PolicyModel): Policy = {
-    val dbPolicy = MongoDBPolicy.fromOther(policy, new ObjectId().toStringMongod)
-    MongoDBStore.insertObject(dbPolicy, policies, MongoDBStore.jsonSupportToDBObject)
+  def loadPolicyAt(atMillis: Long): Option[PolicyMsg] = {
+    // FIXME Inefficient
+    var _policies = immutable.TreeSet[PolicyMsg]()(OrderingHelpers.DefaultPolicyMsgOrdering)
+    foreachPolicy(_policies += _)
+    _policies.to(MessageFactory.newDummyPolicyMsgAt(atMillis)).lastOption
   }
 
-  def loadValidPolicyAt(atMillis: Long): Option[Policy] = {
-    throw new UnsupportedOperationException
-  }
+  def loadSortedPoliciesWithin(fromMillis: Long, toMillis: Long): immutable.SortedMap[Timeslot, PolicyMsg] = {
+    // FIXME Inefficient
+    var _policies = immutable.TreeSet[PolicyMsg]()(OrderingHelpers.DefaultPolicyMsgOrdering)
+    foreachPolicy(_policies += _)
 
-  def loadAndSortPoliciesWithin(fromMillis: Long, toMillis: Long): SortedMap[Timeslot, Policy] = {
-    throw new UnsupportedOperationException
+    immutable.SortedMap(_policies.
+      from(MessageFactory.newDummyPolicyMsgAt(fromMillis)).
+      to(MessageFactory.newDummyPolicyMsgAt(toMillis)).toSeq.
+      map(p ⇒ (Timeslot(p.getValidFromMillis, p.getValidToMillis), p)): _*
+    )
   }
   //-PolicyStore
 }
 
 object MongoDBStore {
-  object JsonNames {
-    final val _id = "_id"
-  }
+  final val JsonNames = gr.grnet.aquarium.util.json.JsonNames
 
-  /**
-   * 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"
+  final val collections = List("resevents","userstates","imevents","policies")
 
-  /**
-   * 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"
+  final val ResourceEventCollection = collections(0)
 
-  /**
-   * 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"
+  final val UserStateCollection = collections(1)
 
-  /**
-   * Collection holding [[gr.grnet.aquarium.policy.PolicyModel]]s.
-   */
-  final val POLICY_COLLECTION = "policies"
+  final val IMEventCollection = collections(2)
 
-  def dbObjectToUserState(dbObj: DBObject): UserState = {
-    UserState.fromJson(JSON.serialize(dbObj))
-  }
+  final val PolicyCollection = collections(3)
 
   def firstResultIfExists[A](cursor: DBCursor, f: DBObject ⇒ A): Option[A] = {
     withCloseable(cursor) { cursor ⇒
@@ -343,92 +430,37 @@ object MongoDBStore {
     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 findOneByAttribute(
+      collection: DBCollection,
+      attributeName: String,
+      attributeValue: String,
+      sortByOpt: Option[DBObject] = None
+  ): Option[DBObject] =  {
+    val query = new BasicDBObject(attributeName, attributeValue)
+    val cursor = sortByOpt match {
+      case None         ⇒ collection find query
+      case Some(sortBy) ⇒ collection find query sort sortBy
     }
-  }
-
-  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
-      }
+    withCloseable(cursor) { cursor ⇒
+      if(cursor.hasNext) Some(cursor.next()) else None
     }
   }
 
-  def storeUserState(userState: UserState, collection: DBCollection) = {
-    storeAny[UserState](userState, collection, ResourceEventNames.userID, _.userID, MongoDBStore.jsonSupportToDBObject)
+  def insertDBObject(dbObj: DBObject, collection: DBCollection) {
+    collection.insert(dbObj, WriteConcern.JOURNAL_SAFE)
   }
 
-  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)
-  }
-
-  def insertObject[A <: AnyRef](obj: A, collection: DBCollection, serializer: A ⇒ DBObject) : A = {
-    collection.insert(serializer apply obj, WriteConcern.JOURNAL_SAFE)
-    obj
+  def findNextPayloadRecord[R <: SpecificRecord](cursor: DBCursor, fresh: R): Option[R] = {
+    for {
+      dbObject <- if(cursor.hasNext) Some(cursor.next()) else None
+      payload = dbObject.get(MongoDBStore.JsonNames.payload).asInstanceOf[Array[Byte]]
+      msg = AvroHelpers.specificRecordOfBytes(payload, fresh)
+    } yield {
+      msg
+    }
   }
 
   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)
+    StdConverters.AllConverters.convertEx[DBObject](jsonSupport)
   }
 }