Skip to content

Commit

Permalink
[DCS-231] DAO interface should not hide underlying exceptions (#50)
Browse files Browse the repository at this point in the history
  • Loading branch information
darroyo-stratio authored Dec 21, 2016
1 parent 437ad62 commit b9195e3
Show file tree
Hide file tree
Showing 9 changed files with 212 additions and 127 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,38 +17,40 @@ package com.stratio.common.utils.components.dao

import com.stratio.common.utils.components.repository.RepositoryComponent

import scala.util.Try

trait DAOComponent[K, V, M] {
self: RepositoryComponent[K, V] =>

val dao: DAO

trait DAO {

def get(id: K) (implicit manifest: Manifest[M]): Option[M] =
repository.get(entity, id).map(entity => fromVtoM(entity))
def get(id: K) (implicit manifest: Manifest[M]): Try[Option[M]] =
repository.get(entity, id).map(_.map(entity => fromVtoM(entity)))

def getAll() (implicit manifest: Manifest[M]): List[M] =
repository.getAll(entity).map(fromVtoM(_))
def getAll() (implicit manifest: Manifest[M]): Try[Seq[M]] =
repository.getAll(entity).map(_.map(fromVtoM))

def count(): Long =
def count(): Try[Long] =
repository.count(entity)

def exists(id: K): Boolean =
def exists(id: K): Try[Boolean] =
repository.exists(entity, id)

def create(id: K, element: M) (implicit manifest: Manifest[M]): M =
fromVtoM(repository.create(entity, id, fromMtoV(element)))
def create(id: K, element: M) (implicit manifest: Manifest[M]): Try[M] =
repository.create(entity, id, fromMtoV(element)).map(fromVtoM)

def update(id: K, element: M) (implicit manifest: Manifest[M]): Unit =
def update(id: K, element: M) (implicit manifest: Manifest[M]): Try[Unit] =
repository.update(entity, id, fromMtoV(element))

def upsert(id: K, element: M) (implicit manifest: Manifest[M]): M =
fromVtoM(repository.upsert(entity, id, fromMtoV(element)))
def upsert(id: K, element: M) (implicit manifest: Manifest[M]): Try[M] =
repository.upsert(entity, id, fromMtoV(element)).map(fromVtoM)

def delete(id: K): Unit =
def delete(id: K): Try[Unit] =
repository.delete(entity, id)

def deleteAll: Unit =
def deleteAll: Try[Unit] =
repository.deleteAll(entity)

def entity: String
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,30 +15,32 @@
*/
package com.stratio.common.utils.components.repository

import scala.util.Try

trait RepositoryComponent[K, V] {

val repository: Repository

trait Repository {

def get(entity: String, id: K): Option[V]
def get(entity: String, id: K): Try[Option[V]]

def getAll(entity: String): List[V]
def getAll(entity: String): Try[Seq[V]]

def getNodes(entity: String): List[K]
def getNodes(entity: String): Try[Seq[K]]

def count(entity: String): Long
def count(entity: String): Try[Long]

def exists(entity: String, id: K): Boolean
def exists(entity: String, id: K): Try[Boolean]

def create(entity: String, id: K, element: V): V
def create(entity: String, id: K, element: V): Try[V]

def upsert(entity: String, id: K, element: V): V
def upsert(entity: String, id: K, element: V): Try[V]

def update(entity: String, id: K, element: V): Unit
def update(entity: String, id: K, element: V): Try[Unit]

def delete(entity: String, id: K): Unit
def delete(entity: String, id: K): Try[Unit]

def deleteAll(entity: String): Unit
def deleteAll(entity: String): Try[Unit]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
*/
package com.stratio.common.utils.components.repository.impl

import java.util.NoSuchElementException
import java.util.concurrent.ConcurrentHashMap

import com.stratio.common.utils.components.config.ConfigComponent
Expand All @@ -31,6 +30,8 @@ import org.json4s.jackson.Serialization.read
import scala.collection.JavaConversions._
import scala.util.{Failure, Success, Try}
import com.stratio.common.utils.components.repository.impl.ZookeeperRepositoryComponent._
import com.stratio.common.utils.functional.TryUtils


trait ZookeeperRepositoryComponent extends RepositoryComponent[String, Array[Byte]] {
self: ConfigComponent with LoggerComponent =>
Expand All @@ -42,67 +43,69 @@ trait ZookeeperRepositoryComponent extends RepositoryComponent[String, Array[Byt
private def curatorClient: CuratorFramework =
ZookeeperRepository.getInstance(getZookeeperConfig)

def get(entity: String, id: String): Option[Array[Byte]] =
def get(entity: String, id: String): Try[Option[Array[Byte]]] =
Try(Option(curatorClient
.getData
.forPath(s"/$entity/$id"))).getOrElse(None)
.forPath(s"/$entity/$id")))

def getAll(entity: String): Try[Seq[Array[Byte]]] = {

def getAll(entity: String): List[Array[Byte]] =
Try(curatorClient
.getChildren
.forPath(s"/$entity").map(get(entity, _).get).toList).getOrElse(List.empty[Array[Byte]])
.forPath(s"/$entity")).flatMap(entityIds =>
TryUtils.sequence(entityIds.map(get(entity, _).map(_.get)))
)
}


def getNodes(entity: String): List[String] =
def getNodes(entity: String): Try[Seq[String]] =
Try(curatorClient
.getChildren
.forPath(s"/$entity").toList).getOrElse(List.empty[String])
.forPath(s"/$entity"))

def count(entity: String): Long =
def count(entity: String): Try[Long] =
Try(curatorClient
.getChildren
.forPath(s"/$entity").size.toLong).getOrElse(0L)
.forPath(s"/$entity").size.toLong)

def exists(entity: String, id: String): Boolean =
def exists(entity: String, id: String): Try[Boolean] =
Try(Option(curatorClient
.checkExists()
.forPath(s"/$entity/$id"))
).getOrElse(None).isDefined
).map(_.isDefined)

def create(entity: String, id: String, element: Array[Byte]): Array[Byte] = {
Try(curatorClient
.create()
.creatingParentsIfNeeded()
.forPath(s"/$entity/$id", element))
def create(entity: String, id: String, element: Array[Byte]): Try[Array[Byte]] =
Try(
curatorClient
.create()
.creatingParentsIfNeeded()
.forPath(s"/$entity/$id", element)
).flatMap( _ => get(entity, id).map(_.get))

get(entity, id)
.getOrElse(throw new NoSuchElementException(s"Something were wrong when retrieving element $id after create"))
}

def upsert(entity: String, id: String, element: Array[Byte]): Array[Byte] =
if (!exists(entity, id)) create(entity, id, element)
else {
update(entity, id, element)
get(entity, id)
.getOrElse(throw new NoSuchElementException(s"Something were wrong when retrieving element $id after create"))
def upsert(entity: String, id: String, element: Array[Byte]): Try[Array[Byte]] =
exists(entity, id).flatMap {
case false => create(entity, id, element)
case true => update(entity, id, element).flatMap(_ => get(entity, id).map(_.get))
}

def update(entity: String, id: String, element: Array[Byte]): Unit =
def update(entity: String, id: String, element: Array[Byte]): Try[Unit] =
Try(curatorClient
.setData()
.forPath(s"/$entity/$id", element)
).getOrElse(throw new ZookeeperRepositoryException(s"Something were wrong when updating element $id"))
)

def delete(entity: String, id: String): Unit =
def delete(entity: String, id: String): Try[Unit] =
Try(curatorClient
.delete()
.forPath(s"/$entity/$id")
).getOrElse(throw new ZookeeperRepositoryException(s"Something were wrong when deleting element $id"))
)

def deleteAll(entity: String): Unit =
def deleteAll(entity: String): Try[Unit] =
Try(curatorClient
.delete().deletingChildrenIfNeeded()
.forPath(s"/$entity")
).getOrElse(throw new ZookeeperRepositoryException(s"Something were wrong when deleting all"))
)

def getZookeeperConfig: Config = {
config.getConfig(path.getOrElse(ConfigZookeeper))
Expand Down
31 changes: 31 additions & 0 deletions src/main/scala/com/stratio/common/utils/functional/TryUtils.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Copyright (C) 2015 Stratio (http://stratio.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.stratio.common.utils.functional

import scala.util.{Failure, Success, Try}

object TryUtils {

def sequence[T](s: Seq[Try[T]]): Try[Seq[T]] = {
def recSequence(s: Seq[Try[T]], acc: Seq[T]): Try[Seq[T]] =
s.headOption map {
case Failure(cause) => Failure(cause)
case Success(v) => recSequence(s.tail, v +: acc)
} getOrElse Success(acc reverse)
recSequence(s, Seq.empty)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ import com.stratio.common.utils.components.repository.DummyRepositoryComponent
import org.json4s.DefaultFormats
import org.scalatest.{Matchers, WordSpec}

import scala.util.{Success, Try}

class DaoComponentTest extends WordSpec with Matchers {

trait DummyDAOComponentContext extends DummyDAOComponent {
Expand All @@ -40,11 +42,11 @@ class DaoComponentTest extends WordSpec with Matchers {
"check if a value exists" should {

"return true if the value exists" in new DummyDAOComponentContext {
dao.exists("key1") should be(true)
dao.exists("key1") should matchPattern {case Success(true) => }
}

"return false if the value doesn't exist" in new DummyDAOComponentContext {
dao.exists("keyNotFound") should be(false)
dao.exists("keyNotFound") should matchPattern {case Success(false) => }
}
}

Expand Down Expand Up @@ -72,7 +74,7 @@ class DaoComponentTest extends WordSpec with Matchers {

"return false if the operation is not successful" in new DummyDAOComponentContext {
dao.delete("keyNotFound")
dao.getAll().size should be(3)
dao.getAll().map(_.size) should matchPattern {case Success(3) => }
}
}

Expand All @@ -86,7 +88,7 @@ class DaoComponentTest extends WordSpec with Matchers {

"return false if the operation is not successful" in new DummyDAOComponentContext {
dao.update("keyNotFound", DummyModel("newValue"))
dao.getAll().size should be(3)
dao.getAll().map(_.size) should matchPattern {case Success(3) => }
}
}

Expand All @@ -104,7 +106,7 @@ class DaoComponentTest extends WordSpec with Matchers {

"return false if the operation is not successful" in new DummyDAOComponentContext {
dao.update("keyNotFound", DummyModel("newValue"))
dao.getAll().size should be(3)
dao.getAll().map(_.size) should matchPattern {case Success(3) => }
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
package com.stratio.common.utils.components.repository

import scala.collection.mutable
import scala.util.{Failure, Success, Try}

trait DummyRepositoryComponent extends RepositoryComponent[String, String] {

Expand All @@ -32,55 +33,49 @@ trait DummyRepositoryComponent extends RepositoryComponent[String, String] {

class DummyRepository() extends Repository {

def get(entity:String, id: String): Option[String] =
memoryMap.get(entity).flatMap(_.get(id))
def get(entity: String, id: String): Try[Option[String]] =
Try(memoryMap.get(entity).flatMap(_.get(id)))

def getAll(entity:String): List[String] =
memoryMap.get(entity) match {
case Some(map: mutable.Map[String,String]) => {
map.values.toList.sortBy(x => x)
}
case _ => List.empty[String]
}
def getAll(entity: String): Try[Seq[String]] =
Try(memoryMap.get(entity).map(_.values.toList.sortBy(identity)).getOrElse(Seq.empty))

def getNodes(entity:String): List[String] =
memoryMap.get(entity) match {
case Some(map: mutable.Map[String,String]) => {
map.keys.toList.sortBy(x => x)
}
case _ => List.empty[String]
}
def getNodes(entity: String): Try[Seq[String]] =
Try(memoryMap.get(entity).map(_.keys.toList.sortBy(identity)).getOrElse(Seq.empty))

def count(entity: String): Long =
memoryMap.get(entity) match {
case Some(value) => value.size
case None => 0
}
def count(entity: String): Try[Long] =
Try(memoryMap.get(entity).map(_.size).getOrElse(0).toLong)

def exists(entity:String, id: String): Boolean =
memoryMap.get(entity) match {
case Some(value) => value.get(id).isDefined
case None => false
}
def exists(entity: String, id: String): Try[Boolean] =
Try(memoryMap.exists { case (key, entityMap) =>
key == entity && entityMap.keys.toList.contains(id)
})

def create(entity:String, id: String, element: String): String = {
if (!exists(entity, id)) {
memoryMap.put(entity, mutable.Map(id -> element))
def create(entity:String, id: String, element: String): Try[String] = {
exists(entity, id).foreach {
case false => memoryMap.put(entity, mutable.Map(id -> element))
case true => ()
}
element
Success(element)
}

override def upsert(entity: String, id: String, element: String): String = {
override def upsert(entity: String, id: String, element: String): Try[String] = {
memoryMap.put(entity, mutable.Map(id -> element))
element
Success(element)
}

def update(entity:String, id: String, element: String): Unit =
if (exists(entity, id)) memoryMap.put(entity, mutable.Map(id -> element))
def update(entity: String, id: String, element: String): Try[Unit] =
exists(entity, id).map {
case false => ()
case true => memoryMap.put(entity, mutable.Map(id -> element))
}

def delete(entity:String, id: String): Unit =
if (exists(entity, id)) memoryMap.get(entity).get.remove(id)
def delete(entity:String, id: String): Try[Unit] = {
exists(entity, id).map{
case false => ()
case true => memoryMap(entity).remove(id)
}
}

def deleteAll(entity:String): Unit = memoryMap.remove(entity)
def deleteAll(entity:String): Try[Unit] = Try(memoryMap.remove(entity))
}
}
Loading

0 comments on commit b9195e3

Please sign in to comment.