From f33b66cf2c6a3f07ba746200e7cfbfa0b8aa1da9 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 21 Apr 2011 18:12:16 -0700 Subject: [PATCH 01/16] wip --- .../com/twitter/gizzard/GizzardServer.scala | 6 +- .../twitter/gizzard/config/NameServer.scala | 16 +- .../gizzard/nameserver/LoadBalancer.scala | 29 +-- .../gizzard/nameserver/MemoryShard.scala | 2 +- .../gizzard/nameserver/NameServer.scala | 10 +- .../nameserver/ReadWriteShardAdapter.scala | 74 ++++---- .../twitter/gizzard/nameserver/Shard.scala | 4 +- .../gizzard/nameserver/ShardRepository.scala | 49 +++-- .../twitter/gizzard/nameserver/SqlShard.scala | 2 +- .../twitter/gizzard/scheduler/CopyJob.scala | 20 +- .../twitter/gizzard/scheduler/RepairJob.scala | 36 ++-- .../gizzard/shards/BlackHoleShard.scala | 25 ++- .../twitter/gizzard/shards/BlockedShard.scala | 26 ++- .../gizzard/shards/FailingOverShard.scala | 15 +- .../gizzard/shards/ReadOnlyShard.scala | 27 +-- .../gizzard/shards/ReadWriteShard.scala | 85 ++++++++- .../gizzard/shards/ReplicatingShard.scala | 176 +++++++----------- .../twitter/gizzard/shards/ShardFactory.scala | 4 +- .../gizzard/shards/WriteOnlyShard.scala | 28 +-- .../gizzard/test/FakeReadWriteShard.scala | 14 +- .../gizzard/thrift/ManagerService.scala | 2 +- 21 files changed, 326 insertions(+), 324 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/GizzardServer.scala b/src/main/scala/com/twitter/gizzard/GizzardServer.scala index 10386062..b39adeff 100644 --- a/src/main/scala/com/twitter/gizzard/GizzardServer.scala +++ b/src/main/scala/com/twitter/gizzard/GizzardServer.scala @@ -5,13 +5,11 @@ import com.twitter.util.TimeConversions._ import net.lag.logging.Logger import nameserver.{NameServer, BasicShardRepository} import scheduler.{CopyJobFactory, JobScheduler, JsonJob, JobConsumer, PrioritizingJobScheduler, ReplicatingJsonCodec, RepairJobFactory} -import shards.{Shard, ReadWriteShard} import config.{GizzardServer => ServerConfig} -abstract class GizzardServer[S <: Shard](config: ServerConfig) { +abstract class GizzardServer[S](config: ServerConfig) { - def readWriteShardAdapter: ReadWriteShard[S] => S def copyFactory: CopyJobFactory[S] def repairFactory: RepairJobFactory[S] = null def diffFactory: RepairJobFactory[S] = null @@ -30,7 +28,7 @@ abstract class GizzardServer[S <: Shard](config: ServerConfig) { // nameserver/shard wiring val replicationFuture: Option[Future] = None - lazy val shardRepo = new BasicShardRepository[S](readWriteShardAdapter, replicationFuture) + lazy val shardRepo = new BasicShardRepository[S](replicationFuture) lazy val nameServer = config.nameServer(shardRepo) diff --git a/src/main/scala/com/twitter/gizzard/config/NameServer.scala b/src/main/scala/com/twitter/gizzard/config/NameServer.scala index a567c984..b032ecf3 100644 --- a/src/main/scala/com/twitter/gizzard/config/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/config/NameServer.scala @@ -52,7 +52,7 @@ trait NameServer { def replicas: Seq[Replica] var jobRelay: JobRelay = new JobRelay - protected def getMappingFunction: (Long => Long) = { + protected def getMappingFunction(): (Long => Long) = { mappingFunction match { case gizzard.config.ByteSwapper => nameserver.ByteSwapper case gizzard.config.Identity => { n => n } @@ -60,13 +60,15 @@ trait NameServer { } } - def apply[S <: shards.Shard](shardRepository: nameserver.ShardRepository[S]) = { - val replicaShards = replicas.map(_.apply()) + def apply[T](shardRepository: nameserver.ShardRepository[T]) = { + val replicaInfo = new shards.ShardInfo("", "", "") + val replicaNodes = replicas map { r => new shards.LeafRoutingNode(replicaInfo, 1, r()) } + val shardInfo = new shards.ShardInfo("com.twitter.gizzard.nameserver.ReplicatingShard", "", "") - val loadBalancer = new nameserver.LoadBalancer(replicaShards) - val shard = new nameserver.ReadWriteShardAdapter( - new shards.ReplicatingShard(shardInfo, 0, replicaShards, loadBalancer, None)) + val loadBalancer = new nameserver.LoadBalancer(replicaNodes) + val replicating = new shards.ReplicatingShard(shardInfo, 0, replicaNodes, loadBalancer, None) + val shard = new nameserver.ReadWriteShardAdapter(replicating) - new nameserver.NameServer(shard, shardRepository, jobRelay(), getMappingFunction) + new nameserver.NameServer(shard, shardRepository, jobRelay(), getMappingFunction()) } } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/LoadBalancer.scala b/src/main/scala/com/twitter/gizzard/nameserver/LoadBalancer.scala index bbbeb2b1..048f19d2 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/LoadBalancer.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/LoadBalancer.scala @@ -3,20 +3,21 @@ package nameserver import scala.util.Random import scala.collection.mutable.ArrayBuffer +import com.twitter.gizzard.shards.RoutingNode -class LoadBalancer[ConcreteShard <: shards.Shard]( - random: Random, - replicas: Seq[ConcreteShard]) - extends (() => Seq[ConcreteShard]) { +class LoadBalancer[T]( + random: Random, + replicas: Seq[RoutingNode[T]]) +extends (() => Seq[RoutingNode[T]]) { - def this(replicas: Seq[ConcreteShard]) = this(new Random, replicas) + def this(replicas: Seq[RoutingNode[T]]) = this(new Random, replicas) def apply() = sort(replicas) - protected def sort(replicas: Seq[ConcreteShard]): List[ConcreteShard] = + protected def sort(replicas: Seq[RoutingNode[T]]): List[RoutingNode[T]] = sort(replicas.foldLeft(0)(_ + _.weight), replicas) - protected def sort(totalWeight: Int, replicas: Seq[ConcreteShard]): List[ConcreteShard] = replicas match { + protected def sort(totalWeight: Int, replicas: Seq[RoutingNode[T]]): List[RoutingNode[T]] = replicas match { case Seq() => Nil case Seq(first, rest @ _*) => val remainingWeight = totalWeight - first.weight @@ -31,12 +32,12 @@ class LoadBalancer[ConcreteShard <: shards.Shard]( } } -class FailingOverLoadBalancer[ConcreteShard <: shards.Shard]( - random: Random, - replicas: Seq[ConcreteShard]) - extends LoadBalancer[ConcreteShard](random, replicas) { +class FailingOverLoadBalancer[T]( + random: Random, + replicas: Seq[RoutingNode[T]]) +extends LoadBalancer[T](random, replicas) { - def this(replicas: Seq[ConcreteShard]) = this(new Random, replicas) + def this(replicas: Seq[RoutingNode[T]]) = this(new Random, replicas) val keepWarmFalloverRate = 0.01 @@ -53,8 +54,8 @@ class FailingOverLoadBalancer[ConcreteShard <: shards.Shard]( } } - private def randomize(replicas: Seq[ConcreteShard]) = { - val buffer = new ArrayBuffer[ConcreteShard] + private def randomize(replicas: Seq[RoutingNode[T]]) = { + val buffer = new ArrayBuffer[RoutingNode[T]] buffer ++= replicas def swap(a: Int, b: Int) { diff --git a/src/main/scala/com/twitter/gizzard/nameserver/MemoryShard.scala b/src/main/scala/com/twitter/gizzard/nameserver/MemoryShard.scala index 7f2345ab..23d32c1b 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/MemoryShard.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/MemoryShard.scala @@ -39,7 +39,7 @@ class MemoryShard extends Shard { dumpStructure(tableIds.toSeq) } - def createShard[S <: shards.Shard](shardInfo: ShardInfo, repository: ShardRepository[S]) { + def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) { find(shardInfo) match { case Some(x) => if (x.className != shardInfo.className || diff --git a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala index 727c2be1..3c9dc9c9 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala @@ -39,9 +39,9 @@ object TreeUtils { } } -class NameServer[S <: shards.Shard]( +class NameServer[T]( nameServerShard: Shard, - shardRepository: ShardRepository[S], + shardRepository: ShardRepository[T], jobRelayFactory: JobRelayFactory, val mappingFunction: Long => Long) { @@ -123,7 +123,7 @@ class NameServer[S <: shards.Shard]( log.info("Loading name server configuration is done.") } - def findShardById(id: ShardId, weight: Int): S = { + def findShardById(id: ShardId, weight: Int): RoutingNode[T] = { val (shardInfo, downwardLinks) = shardInfos.get(id).map { info => // either pull shard and links from our internal data structures... (info, getChildren(id)) @@ -138,7 +138,7 @@ class NameServer[S <: shards.Shard]( } @throws(classOf[NonExistentShard]) - def findShardById(id: ShardId): S = findShardById(id, 1) + def findShardById(id: ShardId): RoutingNode[T] = findShardById(id, 1) def findCurrentForwarding(tableId: Int, id: Long) = { if(forwardings == null) throw new NameserverUninitialized @@ -184,7 +184,7 @@ class NameServer[S <: shards.Shard]( ids.map(getRootShardIds).reduceLeft((s1, s2) => s1.filter(s2.contains)).toSeq.headOption } - @throws(classOf[shards.ShardException]) def createShard[S <: shards.Shard](shardInfo: ShardInfo, repository: ShardRepository[S]) = nameServerShard.createShard(shardInfo, repository) + @throws(classOf[shards.ShardException]) def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = nameServerShard.createShard(shardInfo, repository) @throws(classOf[shards.ShardException]) def getShard(id: ShardId) = nameServerShard.getShard(id) @throws(classOf[shards.ShardException]) def deleteShard(id: ShardId) = nameServerShard.deleteShard(id) @throws(classOf[shards.ShardException]) def addLink(upId: ShardId, downId: ShardId, weight: Int) = nameServerShard.addLink(upId, downId, weight) diff --git a/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala b/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala index 0026e780..7a4da80f 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala @@ -1,46 +1,46 @@ package com.twitter.gizzard package nameserver -import shards.{ReadWriteShard, ShardId, ShardInfo, Busy} - -class ReadWriteShardAdapter(shard: ReadWriteShard[Shard]) extends shards.ReadWriteShardAdapter(shard) with Shard { - def getBusyShards() = shard.readOperation(_.getBusyShards()) - def getForwarding(tableId: Int, baseId: Long) = shard.readOperation(_.getForwarding(tableId, baseId)) - def getForwardingForShard(id: ShardId) = shard.readOperation(_.getForwardingForShard(id)) - def getForwardings() = shard.readOperation(_.getForwardings()) - def getForwardingsForTableIds(tableIds: Seq[Int]): Seq[Forwarding] = shard.readOperation(_.getForwardingsForTableIds(tableIds)) - def getShard(id: ShardId) = shard.readOperation(_.getShard(id)) - def listUpwardLinks(id: ShardId) = shard.readOperation(_.listUpwardLinks(id)) - def listDownwardLinks(id: ShardId) = shard.readOperation(_.listDownwardLinks(id)) - def listLinks() = shard.readOperation(_.listLinks()) - def listShards() = shard.readOperation(_.listShards()) - def shardsForHostname(hostname: String) = shard.readOperation(_.shardsForHostname(hostname)) - def listHostnames() = shard.readOperation(_.listHostnames()) - def listTables() = shard.readOperation(_.listTables()) - - def currentState() = shard.readOperation(_.currentState()) - - def createShard[S <: shards.Shard](shardInfo: ShardInfo, repository: ShardRepository[S]) = shard.writeOperation(_.createShard(shardInfo, repository)) - def deleteShard(id: ShardId) = shard.writeOperation(_.deleteShard(id)) - def markShardBusy(id: ShardId, busy: Busy.Value) = shard.writeOperation(_.markShardBusy(id, busy)) - def addLink(upId: ShardId, downId: ShardId, weight: Int) = shard.writeOperation(_.addLink(upId, downId, weight)) - def removeLink(upId: ShardId, downId: ShardId) = shard.writeOperation(_.removeLink(upId, downId)) - def replaceForwarding(oldId: ShardId, newId: ShardId) = shard.writeOperation(_.replaceForwarding(oldId, newId)) - def setForwarding(forwarding: Forwarding) = shard.writeOperation(_.setForwarding(forwarding)) - def removeForwarding(forwarding: Forwarding) = shard.writeOperation(_.removeForwarding(forwarding)) - def reload() = shard.writeOperation(_.reload()) - def rebuildSchema() = shard.writeOperation(_.rebuildSchema()) +import shards.{RoutingNode, ShardId, ShardInfo, Busy} + +class ReadWriteShardAdapter(node: RoutingNode[Shard]) extends Shard { + def getBusyShards() = node.readOperation(_.getBusyShards()) + def getForwarding(tableId: Int, baseId: Long) = node.readOperation(_.getForwarding(tableId, baseId)) + def getForwardingForShard(id: ShardId) = node.readOperation(_.getForwardingForShard(id)) + def getForwardings() = node.readOperation(_.getForwardings()) + def getForwardingsForTableIds(tableIds: Seq[Int]): Seq[Forwarding] = node.readOperation(_.getForwardingsForTableIds(tableIds)) + def getShard(id: ShardId) = node.readOperation(_.getShard(id)) + def listUpwardLinks(id: ShardId) = node.readOperation(_.listUpwardLinks(id)) + def listDownwardLinks(id: ShardId) = node.readOperation(_.listDownwardLinks(id)) + def listLinks() = node.readOperation(_.listLinks()) + def listShards() = node.readOperation(_.listShards()) + def shardsForHostname(hostname: String) = node.readOperation(_.shardsForHostname(hostname)) + def listHostnames() = node.readOperation(_.listHostnames()) + def listTables() = node.readOperation(_.listTables()) + + def currentState() = node.readOperation(_.currentState()) + + def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = node.writeOperation(_.createShard(shardInfo, repository)) + def deleteShard(id: ShardId) = node.writeOperation(_.deleteShard(id)) + def markShardBusy(id: ShardId, busy: Busy.Value) = node.writeOperation(_.markShardBusy(id, busy)) + def addLink(upId: ShardId, downId: ShardId, weight: Int) = node.writeOperation(_.addLink(upId, downId, weight)) + def removeLink(upId: ShardId, downId: ShardId) = node.writeOperation(_.removeLink(upId, downId)) + def replaceForwarding(oldId: ShardId, newId: ShardId) = node.writeOperation(_.replaceForwarding(oldId, newId)) + def setForwarding(forwarding: Forwarding) = node.writeOperation(_.setForwarding(forwarding)) + def removeForwarding(forwarding: Forwarding) = node.writeOperation(_.removeForwarding(forwarding)) + def reload() = node.writeOperation(_.reload()) + def rebuildSchema() = node.writeOperation(_.rebuildSchema()) // Remote Host Cluster Management - def addRemoteHost(h: Host) = shard.writeOperation(_.addRemoteHost(h)) - def removeRemoteHost(h: String, p: Int) = shard.writeOperation(_.removeRemoteHost(h, p)) - def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = shard.writeOperation(_.setRemoteHostStatus(h, p, s)) - def setRemoteClusterStatus(c: String, s: HostStatus.Value) = shard.writeOperation(_.setRemoteClusterStatus(c, s)) + def addRemoteHost(h: Host) = node.writeOperation(_.addRemoteHost(h)) + def removeRemoteHost(h: String, p: Int) = node.writeOperation(_.removeRemoteHost(h, p)) + def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = node.writeOperation(_.setRemoteHostStatus(h, p, s)) + def setRemoteClusterStatus(c: String, s: HostStatus.Value) = node.writeOperation(_.setRemoteClusterStatus(c, s)) - def getRemoteHost(h: String, p: Int) = shard.readOperation(_.getRemoteHost(h, p)) - def listRemoteClusters() = shard.readOperation(_.listRemoteClusters()) - def listRemoteHosts() = shard.readOperation(_.listRemoteHosts()) - def listRemoteHostsInCluster(c: String) = shard.readOperation(_.listRemoteHostsInCluster(c)) + def getRemoteHost(h: String, p: Int) = node.readOperation(_.getRemoteHost(h, p)) + def listRemoteClusters() = node.readOperation(_.listRemoteClusters()) + def listRemoteHosts() = node.readOperation(_.listRemoteHosts()) + def listRemoteHostsInCluster(c: String) = node.readOperation(_.listRemoteHostsInCluster(c)) } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/Shard.scala b/src/main/scala/com/twitter/gizzard/nameserver/Shard.scala index 29713017..0f4a8bf6 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/Shard.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/Shard.scala @@ -4,8 +4,8 @@ package nameserver import scala.collection.mutable import shards._ -trait Shard extends shards.Shard { - @throws(classOf[shards.ShardException]) def createShard[S <: shards.Shard](shardInfo: ShardInfo, repository: ShardRepository[S]) +trait Shard { + @throws(classOf[shards.ShardException]) def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) @throws(classOf[shards.ShardException]) def deleteShard(id: ShardId) @throws(classOf[shards.ShardException]) def addLink(upId: ShardId, downId: ShardId, weight: Int) @throws(classOf[shards.ShardException]) def removeLink(upId: ShardId, downId: ShardId) diff --git a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala index 838d26cf..c3d38d76 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala @@ -2,18 +2,22 @@ package com.twitter.gizzard package nameserver import scala.collection.mutable -import shards.{ShardInfo, ShardFactory} -import com.twitter.util.Duration +import shards._ -class ShardRepository[S <: shards.Shard] { - private val shardFactories = mutable.Map.empty[String, ShardFactory[S]] +class ShardRepository[T] { + private val nodeFactories = mutable.Map[String, RoutingNodeFactory[T]]() - def +=(item: (String, ShardFactory[S])) { - shardFactories += item + def +=(item: (String, ShardFactory[T])) { + val (className, shardFactory) = item + nodeFactories += (className -> new LeafRoutingNodeFactory(shardFactory)) } - def find(shardInfo: ShardInfo, weight: Int, children: Seq[S]) = { + def addRoutingNode(className: String, factory: RoutingNodeFactory[T]) { + nodeFactories += (className -> factory) + } + + def find(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { factory(shardInfo.className).instantiate(shardInfo, weight, children) } @@ -22,38 +26,33 @@ class ShardRepository[S <: shards.Shard] { } def factory(className: String) = { - shardFactories.get(className).getOrElse { - val classes = shardFactories.keySet - val message = "No such class: " + className + "\nValid classes:\n" + classes + nodeFactories.get(className) getOrElse { + val message = "No such class: " + className + "\nValid classes:\n" + nodeFactories.keySet throw new NoSuchElementException(message) } } - override def toString() = { - "ShardRepository(" + shardFactories.toString + ")" - } + override def toString() = "ShardRepository(" + nodeFactories.toString + ")" } /** * A ShardRepository that is pre-seeded with read-only, write-only, replicating, and blocked * shard types. */ -class BasicShardRepository[S <: shards.Shard](constructor: shards.ReadWriteShard[S] => S, - replicationFuture: Option[Future]) - extends ShardRepository[S] { +class BasicShardRepository[T](replicationFuture: Option[Future]) +extends ShardRepository[T] { setupPackage("com.twitter.gizzard.shards") setupPackage("") def setupPackage(packageName: String) { - val packageNameDot = if (packageName == "") packageName else (packageName + ".") - this += (packageNameDot + "ReadOnlyShard" -> new shards.ReadOnlyShardFactory(constructor)) - this += (packageNameDot + "BlockedShard" -> new shards.BlockedShardFactory(constructor)) - this += (packageNameDot + "WriteOnlyShard" -> new shards.WriteOnlyShardFactory(constructor)) - this += (packageNameDot + "BlackHoleShard" -> new shards.BlackHoleShardFactory(constructor)) - this += (packageNameDot + "ReplicatingShard" -> - new shards.ReplicatingShardFactory(constructor, replicationFuture)) - this += (packageNameDot + "FailingOverShard" -> - new shards.FailingOverShardFactory(constructor, replicationFuture)) + val prefix = if (packageName == "") packageName else packageName + "." + + addRoutingNode(prefix + "ReadOnlyShard", new shards.ReadOnlyShardFactory[T]) + addRoutingNode(prefix + "BlockedShard", new shards.BlockedShardFactory[T]) + addRoutingNode(prefix + "WriteOnlyShard", new shards.WriteOnlyShardFactory[T]) + addRoutingNode(prefix + "BlackHoleShard", new shards.BlackHoleShardFactory[T]) + addRoutingNode(prefix + "ReplicatingShard", new shards.ReplicatingShardFactory[T](replicationFuture)) + addRoutingNode(prefix + "FailingOverShard", new shards.FailingOverShardFactory[T](replicationFuture)) } } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/SqlShard.scala b/src/main/scala/com/twitter/gizzard/nameserver/SqlShard.scala index 9e980be8..a7954270 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/SqlShard.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/SqlShard.scala @@ -100,7 +100,7 @@ class SqlShard(queryEvaluator: QueryEvaluator) extends nameserver.Shard { // Forwardings/Shard Management Write Methods - def createShard[S <: shards.Shard](shardInfo: ShardInfo, repository: ShardRepository[S]) { + def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) { try { queryEvaluator.transaction { transaction => transaction.selectOne("SELECT * FROM shards WHERE table_prefix = ? AND hostname = ?", diff --git a/src/main/scala/com/twitter/gizzard/scheduler/CopyJob.scala b/src/main/scala/com/twitter/gizzard/scheduler/CopyJob.scala index 39f548f6..7a6739d3 100644 --- a/src/main/scala/com/twitter/gizzard/scheduler/CopyJob.scala +++ b/src/main/scala/com/twitter/gizzard/scheduler/CopyJob.scala @@ -5,7 +5,7 @@ import com.twitter.ostrich.Stats import com.twitter.util.TimeConversions._ import net.lag.logging.Logger import nameserver.{NameServer, NonExistentShard} -import shards.{Shard, ShardId, ShardDatabaseTimeoutException, ShardTimeoutException} +import shards.{RoutingNode, ShardId, ShardDatabaseTimeoutException, ShardTimeoutException} object CopyJob { val MIN_COPY = 500 @@ -15,16 +15,16 @@ object CopyJob { * A factory for creating a new copy job (with default count and a starting cursor) from a source * and destination shard ID. */ -trait CopyJobFactory[S <: Shard] extends ((ShardId, ShardId) => CopyJob[S]) +trait CopyJobFactory[T] extends ((ShardId, ShardId) => CopyJob[T]) /** * A parser that creates a copy job out of json. The basic attributes (source shard ID, destination) * shard ID, and count) are parsed out first, and the remaining attributes are passed to * 'deserialize' to decode any shard-specific data (like a cursor). */ -trait CopyJobParser[S <: Shard] extends JsonJobParser { +trait CopyJobParser[T] extends JsonJobParser { def deserialize(attributes: Map[String, Any], sourceId: ShardId, - destinationId: ShardId, count: Int): CopyJob[S] + destinationId: ShardId, count: Int): CopyJob[T] def apply(attributes: Map[String, Any]): JsonJob = { deserialize(attributes, @@ -43,11 +43,11 @@ trait CopyJobParser[S <: Shard] extends JsonJobParser { * 'copyPage' is called to do the actual data copying. It should return a new CopyJob representing * the next chunk of work to do, or None if the entire copying job is complete. */ -abstract case class CopyJob[S <: Shard](sourceId: ShardId, - destinationId: ShardId, - var count: Int, - nameServer: NameServer[S], - scheduler: JobScheduler) +abstract case class CopyJob[T](sourceId: ShardId, + destinationId: ShardId, + var count: Int, + nameServer: NameServer[T], + scheduler: JobScheduler) extends JsonJob { private val log = Logger.get(getClass.getName) @@ -123,7 +123,7 @@ abstract case class CopyJob[S <: Shard](sourceId: ShardId, "x-copying-" + sourceId + "-" + destinationId } - def copyPage(sourceShard: S, destinationShard: S, count: Int): Option[CopyJob[S]] + def copyPage(sourceShard: RoutingNode[T], destinationShard: RoutingNode[T], count: Int): Option[CopyJob[T]] def serialize: Map[String, Any] } diff --git a/src/main/scala/com/twitter/gizzard/scheduler/RepairJob.scala b/src/main/scala/com/twitter/gizzard/scheduler/RepairJob.scala index 402d3d92..0af051c7 100644 --- a/src/main/scala/com/twitter/gizzard/scheduler/RepairJob.scala +++ b/src/main/scala/com/twitter/gizzard/scheduler/RepairJob.scala @@ -6,7 +6,7 @@ import com.twitter.util.TimeConversions._ import net.lag.logging.Logger import nameserver.{NameServer, NonExistentShard} import collection.mutable.ListBuffer -import shards.{Shard, ShardId, ShardDatabaseTimeoutException, ShardTimeoutException} +import shards.{RoutingNode, ShardId, ShardDatabaseTimeoutException, ShardTimeoutException} trait Repairable[T] { def similar(other: T): Int @@ -20,15 +20,15 @@ object RepairJob { * A factory for creating a new repair job (with default count and a starting cursor) from a source * and destination shard ID. */ -trait RepairJobFactory[S <: Shard] extends (Seq[ShardId] => RepairJob[S]) +trait RepairJobFactory[T] extends (Seq[ShardId] => RepairJob[T]) /** * A parser that creates a repair job out of json. The basic attributes (source shard ID, destination) * shard ID, count) are parsed out first, and the remaining attributes are passed to * 'deserialize' to decode any shard-specific data (like a cursor). */ -trait RepairJobParser[S <: Shard] extends JsonJobParser { - def deserialize(attributes: Map[String, Any], shardIds: Seq[ShardId], count: Int): RepairJob[S] +trait RepairJobParser[T] extends JsonJobParser { + def deserialize(attributes: Map[String, Any], shardIds: Seq[ShardId], count: Int): RepairJob[T] def apply(attributes: Map[String, Any]): JsonJob = { deserialize(attributes, @@ -47,11 +47,11 @@ trait RepairJobParser[S <: Shard] extends JsonJobParser { * 'repair' is called to do the actual data repair. It should return a new Some[RepairJob] representing * the next chunk of work to do, or None if the entire copying job is complete. */ -abstract case class RepairJob[S <: Shard](shardIds: Seq[ShardId], - var count: Int, - nameServer: NameServer[S], - scheduler: PrioritizingJobScheduler, - priority: Int) extends JsonJob { +abstract case class RepairJob[T](shardIds: Seq[ShardId], + var count: Int, + nameServer: NameServer[T], + scheduler: PrioritizingJobScheduler, + priority: Int) extends JsonJob { private val log = Logger.get(getClass.getName) override def shouldReplicate = false @@ -105,35 +105,35 @@ abstract case class RepairJob[S <: Shard](shardIds: Seq[ShardId], "x-"+label.toLowerCase+"-" + shardIds.mkString("-") } - def repair(shards: Seq[S]) + def repair(shards: Seq[RoutingNode[T]]) def serialize: Map[String, Any] } -abstract class MultiShardRepair[S <: Shard, R <: Repairable[R], C <: Any](shardIds: Seq[ShardId], cursor: C, count: Int, - nameServer: NameServer[S], scheduler: PrioritizingJobScheduler, priority: Int) extends RepairJob(shardIds, count, nameServer, scheduler, priority) { +abstract class MultiShardRepair[T, R <: Repairable[R], C <: Any](shardIds: Seq[ShardId], cursor: C, count: Int, + nameServer: NameServer[T], scheduler: PrioritizingJobScheduler, priority: Int) extends RepairJob(shardIds, count, nameServer, scheduler, priority) { private val log = Logger.get(getClass.getName) - def nextRepair(lowestCursor: C): Option[RepairJob[S]] + def nextRepair(lowestCursor: C): Option[RepairJob[T]] - def scheduleDifferent(list: (S, ListBuffer[R], C), tableId: Int, item: R): Unit + def scheduleDifferent(list: (RoutingNode[T], ListBuffer[R], C), tableId: Int, item: R): Unit - def scheduleMissing(list: (S, ListBuffer[R], C), tableId: Int, item: R): Unit + def scheduleMissing(list: (RoutingNode[T], ListBuffer[R], C), tableId: Int, item: R): Unit - def scheduleBulk(otherShards: Seq[S], items: Seq[R]): Unit + def scheduleBulk(otherShards: Seq[RoutingNode[T]], items: Seq[R]): Unit def cursorAtEnd(cursor: C): Boolean def lowestCursor(c1: C, c2: C): C - def smallestList(listCursors: Seq[(S, ListBuffer[R], C)]) = { + def smallestList(listCursors: Seq[(RoutingNode[T], ListBuffer[R], C)]) = { listCursors.filter(!_._2.isEmpty).reduceLeft((list1, list2) => if (list1._2(0).similar(list2._2(0)) < 0) list1 else list2) } def shouldSchedule(original:R, suspect: R): Boolean - def repairListCursor(listCursors: Seq[(S, ListBuffer[R], C)], tableIds: Seq[Int]) = { + def repairListCursor(listCursors: Seq[(RoutingNode[T], ListBuffer[R], C)], tableIds: Seq[Int]) = { if (!tableIds.forall((id) => id == tableIds(0))) { throw new RuntimeException("tableIds didn't match") } else if (nameServer.getCommonShardId(shardIds) == None) { diff --git a/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala b/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala index 8630b4bc..88ab90b5 100644 --- a/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala @@ -1,25 +1,24 @@ package com.twitter.gizzard package shards -class BlackHoleShardFactory[ConcreteShard <: Shard](readWriteShardAdapter: ReadWriteShard[ConcreteShard] => ConcreteShard) extends shards.ShardFactory[ConcreteShard] { - def instantiate(shardInfo: shards.ShardInfo, weight: Int, children: Seq[ConcreteShard]) = - readWriteShardAdapter(new BlackHoleShard[ConcreteShard](shardInfo, weight, children)) - def materialize(shardInfo: shards.ShardInfo) = () + +class BlackHoleShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new BlackHoleShard(shardInfo, weight, children) + } } /** * A shard that refuses all read/write traffic in a silent way. */ -class BlackHoleShard[ConcreteShard <: Shard] - (val shardInfo: ShardInfo, val weight: Int, val children: Seq[Shard]) - extends ReadWriteShard[ConcreteShard] { +class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends PassThroughNode[T](shardInfo, weight, children) { - private def throwException = throw new ShardBlackHoleException(shardInfo.id) + private def exception = throw new ShardBlackHoleException(shardInfo.id) - def readAllOperation[A](method: (ConcreteShard => A)) = Seq[Either[Throwable,A]]() - def readOperation[A](method: (ConcreteShard => A)) = throwException - def writeOperation[A](method: (ConcreteShard => A)) = throwException + override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]]() + override def readOperation[A](method: T => A) = throw exception + override def writeOperation[A](method: T => A) = throw exception - def rebuildableReadOperation[A](method: (ConcreteShard => Option[A]))(rebuild: (ConcreteShard, ConcreteShard) => Unit) = - throwException + override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception } diff --git a/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala b/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala index e767973b..97c3415a 100644 --- a/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala @@ -1,23 +1,21 @@ package com.twitter.gizzard package shards -class BlockedShardFactory[ConcreteShard <: Shard](readWriteShardAdapter: ReadWriteShard[ConcreteShard] => ConcreteShard) extends shards.ShardFactory[ConcreteShard] { - def instantiate(shardInfo: shards.ShardInfo, weight: Int, children: Seq[ConcreteShard]) = - readWriteShardAdapter(new BlockedShard(shardInfo, weight, children)) - def materialize(shardInfo: shards.ShardInfo) = () + +class BlockedShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new BlockedShard(shardInfo, weight, children) + } } -class BlockedShard[ConcreteShard <: Shard] - (val shardInfo: ShardInfo, val weight: Int, val children: Seq[Shard]) - extends ReadWriteShard[ConcreteShard] { +class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends PassThroughNode[T](shardInfo, weight, children) { - val shard = children.head - private def throwException = throw new ShardRejectedOperationException("shard is offline", shardInfo.id) + private def exception = new ShardRejectedOperationException("shard is offline", shardInfo.id) - def readAllOperation[A](method: (ConcreteShard => A)) = try { throwException } catch { case e => Seq(Left(e)) } - def readOperation[A](method: (ConcreteShard => A)) = throwException - def writeOperation[A](method: (ConcreteShard => A)) = throwException + override def readAllOperation[A](method: T => A) = Seq(Left(exception)) + override def readOperation[A](method: T => A) = throw exception + override def writeOperation[A](method: T => A) = throw exception - def rebuildableReadOperation[A](method: (ConcreteShard => Option[A]))(rebuild: (ConcreteShard, ConcreteShard) => Unit) = - throwException + override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception } diff --git a/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala b/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala index c558b8b3..dcd16163 100644 --- a/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala @@ -17,19 +17,14 @@ import com.twitter.gizzard.nameserver.FailingOverLoadBalancer */ -class FailingOverShardFactory[ConcreteShard <: Shard]( - readWriteShardAdapter: ReadWriteShard[ConcreteShard] => ConcreteShard, - future: Option[Future]) - extends ShardFactory[ConcreteShard] { - - def instantiate(info: shards.ShardInfo, weight: Int, replicas: Seq[ConcreteShard]) = - readWriteShardAdapter(new ReplicatingShard( +class FailingOverShardFactory[T](future: Option[Future]) extends RoutingNodeFactory[T] { + def instantiate(info: shards.ShardInfo, weight: Int, replicas: Seq[RoutingNode[T]]) = { + new ReplicatingShard( info, weight, replicas, new FailingOverLoadBalancer(replicas), future - )) - - def materialize(shardInfo: shards.ShardInfo) = () + ) + } } diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala index 92c5bdc4..32faa746 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala @@ -1,28 +1,17 @@ package com.twitter.gizzard package shards -import scala.collection.mutable - -class ReadOnlyShardFactory[ConcreteShard <: Shard](readWriteShardAdapter: ReadWriteShard[ConcreteShard] => ConcreteShard) extends ShardFactory[ConcreteShard] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[ConcreteShard]) = - readWriteShardAdapter(new ReadOnlyShard(shardInfo, weight, children)) - def materialize(shardInfo: ShardInfo) = () +class ReadOnlyShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new ReadOnlyShard(shardInfo, weight, children) + } } -class ReadOnlyShard[ConcreteShard <: Shard] - (val shardInfo: ShardInfo, val weight: Int, val children: Seq[ConcreteShard]) - extends ReadWriteShard[ConcreteShard] { - - val shard = children.head - - def readAllOperation[A](method: (ConcreteShard => A)) = Seq(try { Right(method(shard)) } catch { case e => Left(e) }) +class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends PassThroughNode[T](shardInfo, weight, children) { - def readOperation[A](method: (ConcreteShard => A)) = method(shard) - - def writeOperation[A](method: (ConcreteShard => A)) = + override def writeOperation[A](method: T => A) = { throw new ShardRejectedOperationException("shard is read-only", shardInfo.id) - - def rebuildableReadOperation[A](method: (ConcreteShard => Option[A]))(rebuild: (ConcreteShard, ConcreteShard) => Unit) = - method(shard) + } } diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala index bb402d51..ace5c493 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala @@ -1,8 +1,83 @@ package com.twitter.gizzard.shards -trait ReadWriteShard[ConcreteShard <: Shard] extends Shard { - def readAllOperation[A](method: (ConcreteShard => A)): Seq[Either[Throwable,A]] - def readOperation[A](method: (ConcreteShard => A)): A - def writeOperation[A](method: (ConcreteShard => A)): A - def rebuildableReadOperation[A](method: (ConcreteShard => Option[A]))(rebuild: (ConcreteShard, ConcreteShard) => Unit): Option[A] +import java.lang.reflect.UndeclaredThrowableException +import java.util.concurrent.{ExecutionException, TimeoutException} + + +abstract class RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]): RoutingNode[T] + def materialize(shardInfo: ShardInfo) {} +} + +abstract class RoutingNode[T] { + def shardInfo: ShardInfo + def weight: Int + def children: Seq[RoutingNode[T]] + + def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] + def readOperation[A](f: T => A): A + def writeOperation[A](f: T => A): A + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] + + def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { + rebuildRead(Nil) { (shard, toRebuild) => + val result = f(shard) + if (!result.isEmpty) toRebuild.foreach(rebuild(shard, _)) + result + } match { + case Left(s) => None + case Right(rv) => Some(rv) + } + } + + protected def normalizeException(ex: Throwable, shardId: ShardId): Option[Throwable] = ex match { + case e: ExecutionException => normalizeException(e.getCause, shardId) + // fondly known as JavaOutrageException + case e: UndeclaredThrowableException => normalizeException(e.getCause, shardId) + case e: ShardBlackHoleException => None + case e: TimeoutException => Some(new ReplicatingShardTimeoutException(shardId, e)) + case e => Some(e) + } +} + + +class PassThroughNode[T](val shardInfo: ShardInfo, val weight: Int, val children: Seq[RoutingNode[T]]) +extends RoutingNode[T] { + + val inner = children.head + + def readAllOperation[A](f: T => A) = inner.readAllOperation(f) + def readOperation[A](f: T => A) = inner.readOperation(f) + def writeOperation[A](f: T => A) = inner.writeOperation(f) + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + inner.rebuildRead(toRebuild)(f) + } +} + + +class LeafRoutingNode[T](val shardInfo: ShardInfo, val weight: Int, shard: T) extends RoutingNode[T] { + val children: Seq[RoutingNode[T]] = Seq() + + def readAllOperation[A](f: T => A) = Seq(try { Right(f(shard)) } catch { case e => Left(e) }) + def readOperation[A](f: T => A) = f(shard) + def writeOperation[A](f: T => A) = f(shard) + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + f(shard, toRebuild) match { + case Some(rv) => Right(rv) + case None => Left(shard :: toRebuild) + } + } +} + +class LeafRoutingNodeFactory[T](shardFactory: ShardFactory[T]) extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new LeafRoutingNode(shardInfo, weight, shardFactory.instantiate(shardInfo)) + } + + override def materialize(shardInfo: ShardInfo) { + shardFactory.materialize(shardInfo) + } } diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index 3d95cebd..86d93612 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -13,148 +13,102 @@ import com.twitter.util.Duration import net.lag.logging.Logger -class ReplicatingShardFactory[S <: Shard]( - readWriteShardAdapter: ReadWriteShard[S] => S, - future: Option[Future]) - extends shards.ShardFactory[S] { - - def instantiate(shardInfo: shards.ShardInfo, weight: Int, replicas: Seq[S]) = - readWriteShardAdapter(new ReplicatingShard( +class ReplicatingShardFactory[T](future: Option[Future]) extends RoutingNodeFactory[T] { + def instantiate(shardInfo: shards.ShardInfo, weight: Int, replicas: Seq[RoutingNode[T]]) = { + new ReplicatingShard( shardInfo, weight, replicas, new LoadBalancer(replicas), future - )) - - def materialize(shardInfo: shards.ShardInfo) = () -} - -class ReplicatingShard[S <: Shard]( - val shardInfo: ShardInfo, - val weight: Int, - val children: Seq[S], - val loadBalancer: (() => Seq[S]), - val future: Option[Future]) - extends ReadWriteShard[S] { - - def readAllOperation[A](method: (S => A)) = fanout(method(_), children) - def readOperation[A](method: (S => A)) = failover(method(_), loadBalancer()) - - def writeOperation[A](method: (S => A)) = { - fanout(method, children).map { - case Left(ex) => throw ex - case Right(result) => result - }.headOption.getOrElse(throw new ShardBlackHoleException(shardInfo.id)) + ) } +} - def rebuildableReadOperation[A](method: (S => Option[A]))(rebuild: (S, S) => Unit) = - rebuildableFailover(method, rebuild, loadBalancer(), Nil, false) +class ReplicatingShard[T]( + val shardInfo: ShardInfo, + val weight: Int, + val children: Seq[RoutingNode[T]], + val loadBalancer: (() => Seq[RoutingNode[T]]), + val future: Option[Future]) +extends RoutingNode[T] { lazy val log = Logger.get - protected def unwrapException(exception: Throwable): Throwable = { - exception match { - case e: ExecutionException => - unwrapException(e.getCause) - case e: UndeclaredThrowableException => - // fondly known as JavaOutrageException - unwrapException(e.getCause) - case e => - e - } - } + def readAllOperation[A](f: T => A) = fanout(children)(_.readAllOperation(f)) - protected def fanoutFuture[A](method: (S => A), replicas: Seq[S], future: Future) = { - val results = new mutable.ArrayBuffer[Either[Throwable,A]]() + def readOperation[A](f: T => A) = failover(loadBalancer())(_.readOperation(f)) - replicas.map { replica => (replica, future(method(replica))) }.foreach { case (replica, futureTask) => + def writeOperation[A](f: T => A) = { + val allResults = fanout(children) { c => try { - results += Right(futureTask.get(future.timeout.inMillis, TimeUnit.MILLISECONDS)) + Seq(Right(c.writeOperation(f))) } catch { - case e: Exception => - unwrapException(e) match { - case e: ShardBlackHoleException => - // nothing. - case e: TimeoutException => - results += Left(new ReplicatingShardTimeoutException(replica.shardInfo.id, e)) - case e => - results += Left(e) - } + case e => normalizeException(e, shardInfo.id).map(Left(_)).toSeq } + } map { + case Left(e) => throw e + case Right(result) => result } - results + allResults.headOption getOrElse { + throw new ShardBlackHoleException(shardInfo.id) + } } - protected def fanoutSerial[A](method: (S => A), replicas: Seq[S]) = { - val results = new mutable.ArrayBuffer[Either[Throwable,A]]() + protected def fanout[A](replicas: Seq[RoutingNode[T]])(f: RoutingNode[T] => Seq[Either[Throwable,A]]) = { + future match { + case None => replicas flatMap f + case Some(future) => { + replicas.map { r => Pair(r, future(f(r))) } flatMap { case (replica, task) => + try { + task.get(future.timeout.inMillis, TimeUnit.MILLISECONDS) + } catch { + case e => normalizeException(e, replica.shardInfo.id).map(Left(_)).toSeq + } + } + } + } + } - replicas.foreach { replica => + protected def failover[A](replicas: Seq[RoutingNode[T]])(f: RoutingNode[T] => A): A = { + replicas foreach { replica => try { - results += Right(method(replica)) + return f(replica) } catch { - case e: ShardBlackHoleException => - // nothing. - case e: TimeoutException => - results += Left(new ReplicatingShardTimeoutException(replica.shardInfo.id, e)) - case e => results += Left(e) + case e: ShardRejectedOperationException => () + case e: ShardException => log.warning(e, "Error on %s: %s", replica.shardInfo.id, e) } } - results - } - - protected def fanout[A](method: (S => A), replicas: Seq[S]) = { - future match { - case None => fanoutSerial(method, replicas) - case Some(f) => fanoutFuture(method, replicas, f) - } + throw new ShardOfflineException(shardInfo.id) } - protected def failover[A](f: S => A, replicas: Seq[S]): A = { - replicas match { - case Seq() => - throw new ShardOfflineException(shardInfo.id) - case Seq(shard, remainder @ _*) => - try { - f(shard) + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] = { + val start: Either[List[T],A] = Left(toRebuild) + var everSuccessful = false + + val rv = (children foldLeft start) { (result, replica) => + result match { + case Right(rv) => return Right(rv) + case Left(toRebuild) => try { + val next = replica.rebuildRead(toRebuild)(f) + everSuccessful = true + next } catch { - case e: ShardRejectedOperationException => - failover(f, remainder) - case e: ShardException => - log.warning(e, "Error on %s: %s", shard.shardInfo.id, e) - failover(f, remainder) + case e: ShardRejectedOperationException => Left(toRebuild) + case e: ShardException => { + log.warning(e, "Error on %s: %s", replica.shardInfo.id, e) + Left(toRebuild) + } } } - } + } - protected def rebuildableFailover[A](f: S => Option[A], rebuild: (S, S) => Unit, - replicas: Seq[S], toRebuild: List[S], - everSuccessful: Boolean): Option[A] = { - replicas match { - case Seq() => - if (everSuccessful) { - None - } else { - throw new ShardOfflineException(shardInfo.id) - } - case Seq(shard, remainder @ _*) => - try { - f(shard) match { - case None => - rebuildableFailover(f, rebuild, remainder, shard :: toRebuild, true) - case Some(answer) => - toRebuild.foreach { destShard => rebuild(shard, destShard) } - Some(answer) - } - } catch { - case e: ShardRejectedOperationException => - rebuildableFailover(f, rebuild, remainder, toRebuild, everSuccessful) - case e: ShardException => - log.warning(e, "Error on %s: %s", shard.shardInfo.id, e) - rebuildableFailover(f, rebuild, remainder, toRebuild, everSuccessful) - } + if (!everSuccessful) { + throw new ShardOfflineException(shardInfo.id) + } else { + rv } } } diff --git a/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala b/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala index 6c6bb920..6c871a33 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala @@ -1,7 +1,7 @@ package com.twitter.gizzard.shards -trait ShardFactory[ConcreteShard <: Shard] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[ConcreteShard]): ConcreteShard +trait ShardFactory[T] { + def instantiate(shardInfo: ShardInfo): T def materialize(shardInfo: ShardInfo) } diff --git a/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala b/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala index f5fc5f94..0175c3a9 100644 --- a/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala @@ -1,27 +1,19 @@ package com.twitter.gizzard package shards -import scala.collection.mutable - -class WriteOnlyShardFactory[ConcreteShard <: Shard](readWriteShardAdapter: ReadWriteShard[ConcreteShard] => ConcreteShard) extends shards.ShardFactory[ConcreteShard] { - def instantiate(shardInfo: shards.ShardInfo, weight: Int, children: Seq[ConcreteShard]) = - readWriteShardAdapter(new WriteOnlyShard(shardInfo, weight, children)) - def materialize(shardInfo: shards.ShardInfo) = () +class WriteOnlyShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: shards.ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new WriteOnlyShard(shardInfo, weight, children) + } } -class WriteOnlyShard[ConcreteShard <: Shard] - (val shardInfo: ShardInfo, val weight: Int, val children: Seq[ConcreteShard]) - extends ReadWriteShard[ConcreteShard] { - - val shard = children.head - private def throwException = throw new ShardRejectedOperationException("shard is write-only", shardInfo.id) - - def readAllOperation[A](method: (ConcreteShard => A)) = try { throwException } catch { case e => Seq(Left(e)) } - def readOperation[A](method: (ConcreteShard => A)) = throwException +class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends PassThroughNode[T](shardInfo, weight, children) { - def writeOperation[A](method: (ConcreteShard => A)) = method(shard) + private def exception = new ShardRejectedOperationException("shard is write-only", shardInfo.id) - def rebuildableReadOperation[A](method: (ConcreteShard => Option[A]))(rebuild: (ConcreteShard, ConcreteShard) => Unit) = - throwException + override def readAllOperation[A](method: T => A) = Seq(Left(exception)) + override def readOperation[A](method: T => A) = throw exception + override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception } diff --git a/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala b/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala index 215e6968..517f22f7 100644 --- a/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala +++ b/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala @@ -1,11 +1,11 @@ package com.twitter.gizzard package test -import shards.{ReadWriteShard, Shard, ShardInfo} +import shards.{RoutingNode, LeafRoutingNode, ShardInfo} -class FakeReadWriteShard[S <: Shard](shard: S, val shardInfo: ShardInfo, val weight: Int, val children: Seq[S]) extends ReadWriteShard[S] { - def readAllOperation[A](method: (S => A)) = Seq(try { Right(method(shard)) } catch { case e => Left(e) }) - def readOperation[A](method: (S => A)): A = method(shard) - def writeOperation[A](method: (S => A)): A = method(shard) - def rebuildableReadOperation[A](method: (S => Option[A]))(rebuild: (S, S) => Unit): Option[A] = method(shard) -} +class FakeReadWriteShard[T]( + shard: T, + shardInfo: ShardInfo, + weight: Int, + override val children: Seq[RoutingNode[T]]) +extends LeafRoutingNode[T](shardInfo, weight, shard) diff --git a/src/main/scala/com/twitter/gizzard/thrift/ManagerService.scala b/src/main/scala/com/twitter/gizzard/thrift/ManagerService.scala index c8a93f52..b50023b8 100644 --- a/src/main/scala/com/twitter/gizzard/thrift/ManagerService.scala +++ b/src/main/scala/com/twitter/gizzard/thrift/ManagerService.scala @@ -17,7 +17,7 @@ import net.lag.logging.Logger import java.util.{List => JList} -class ManagerService[S <: shards.Shard](nameServer: NameServer[S], copier: CopyJobFactory[S], scheduler: PrioritizingJobScheduler, copyScheduler: JobScheduler, repairer: RepairJobFactory[S], repairPriority: Int, differ: RepairJobFactory[S]) extends Manager.Iface { +class ManagerService[S](nameServer: NameServer[S], copier: CopyJobFactory[S], scheduler: PrioritizingJobScheduler, copyScheduler: JobScheduler, repairer: RepairJobFactory[S], repairPriority: Int, differ: RepairJobFactory[S]) extends Manager.Iface { val log = Logger.get(getClass.getName) def wrapEx[A](f: => A): A = try { f } catch { From f3821ab72a92afed8a4516442c74fc9e699c15eb Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Tue, 26 Apr 2011 17:54:28 -0700 Subject: [PATCH 02/16] it works --- .../twitter/gizzard/config/NameServer.scala | 3 +- .../gizzard/nameserver/NameServer.scala | 200 ++++++++++++++---- .../nameserver/ReadWriteShardAdapter.scala | 46 ---- .../proxy/SqlExceptionWrappingProxy.scala | 5 +- .../gizzard/shards/ReadWriteShard.scala | 2 + .../shards/ReadWriteShardAdapter.scala | 8 - .../com/twitter/gizzard/shards/Shard.scala | 8 - .../twitter/gizzard/fake/NestableShard.scala | 4 +- .../gizzard/fake/ReadWriteShardAdapter.scala | 9 +- .../com/twitter/gizzard/fake/Shard.scala | 2 +- .../gizzard/integration/TestServer.scala | 41 ++-- .../gizzard/nameserver/LoadBalancerSpec.scala | 14 +- .../gizzard/nameserver/NameServerSpec.scala | 40 ++-- .../nameserver/ShardRepositorySpec.scala | 29 +-- .../gizzard/nameserver/SqlShardSpec.scala | 17 +- .../gizzard/proxy/ExceptionProxySpec.scala | 5 +- .../gizzard/scheduler_new/CopyJobSpec.scala | 27 ++- .../gizzard/shards/ReplicatingShardSpec.scala | 63 +++--- .../shards/ShardsIntegrationSpec.scala | 14 +- .../thrift/ShardManagerServiceSpec.scala | 10 +- 20 files changed, 299 insertions(+), 248 deletions(-) delete mode 100644 src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/ReadWriteShardAdapter.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/Shard.scala diff --git a/src/main/scala/com/twitter/gizzard/config/NameServer.scala b/src/main/scala/com/twitter/gizzard/config/NameServer.scala index b032ecf3..4f2bb31e 100644 --- a/src/main/scala/com/twitter/gizzard/config/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/config/NameServer.scala @@ -67,8 +67,7 @@ trait NameServer { val shardInfo = new shards.ShardInfo("com.twitter.gizzard.nameserver.ReplicatingShard", "", "") val loadBalancer = new nameserver.LoadBalancer(replicaNodes) val replicating = new shards.ReplicatingShard(shardInfo, 0, replicaNodes, loadBalancer, None) - val shard = new nameserver.ReadWriteShardAdapter(replicating) - new nameserver.NameServer(shard, shardRepository, jobRelay(), getMappingFunction()) + new nameserver.NameServer(replicating, shardRepository, jobRelay(), getMappingFunction()) } } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala index 3c9dc9c9..421085b2 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala @@ -4,7 +4,7 @@ package nameserver import java.util.TreeMap import scala.collection.mutable import com.twitter.util.Time -import com.twitter.util.TimeConversions._ +import com.twitter.conversions.time._ import com.twitter.querulous.StatsCollector import com.twitter.querulous.evaluator.QueryEvaluatorFactory import net.lag.logging.Logger @@ -40,7 +40,7 @@ object TreeUtils { } class NameServer[T]( - nameServerShard: Shard, + nameServerShard: RoutingNode[nameserver.Shard], shardRepository: ShardRepository[T], jobRelayFactory: JobRelayFactory, val mappingFunction: Long => Long) { @@ -59,7 +59,7 @@ class NameServer[T]( @throws(classOf[shards.ShardException]) def createShard(shardInfo: ShardInfo) { - nameServerShard.createShard(shardInfo, shardRepository) + createShard(shardInfo, shardRepository) } def getShardInfo(id: ShardId) = shardInfos(id) @@ -69,14 +69,16 @@ class NameServer[T]( familyTree.getOrElse(id, new mutable.ArrayBuffer[LinkInfo]) } - def dumpStructure(tableIds: Seq[Int]) = nameServerShard.dumpStructure(tableIds) + def dumpStructure(tableIds: Seq[Int]) = nameServerShard.readOperation(_.dumpStructure(tableIds)) + + private def currentState() = nameServerShard.readOperation(_.currentState()) private def recreateInternalShardState() { val newShardInfos = mutable.Map[ShardId, ShardInfo]() val newFamilyTree = mutable.Map[ShardId, mutable.ArrayBuffer[LinkInfo]]() val newForwardings = mutable.Map[Int, TreeMap[Long, ShardInfo]]() - nameServerShard.currentState().foreach { state => + currentState().foreach { state => state.shards.foreach { info => newShardInfos += (info.id -> info) } @@ -109,11 +111,11 @@ class NameServer[T]( def reload() { log.info("Loading name server configuration...") - nameServerShard.reload() + nameServerShard.writeOperation(_.reload()) val newRemoteClusters = mutable.Map[String, List[Host]]() - nameServerShard.listRemoteHosts.foreach { h => + listRemoteHosts.foreach { h => newRemoteClusters += h.cluster -> (h :: newRemoteClusters.getOrElse(h.cluster, Nil)) } @@ -142,7 +144,7 @@ class NameServer[T]( def findCurrentForwarding(tableId: Int, id: Long) = { if(forwardings == null) throw new NameserverUninitialized - val shardInfo = forwardings.get(tableId).flatMap { bySourceIds => + val shardInfo = forwardings.get(tableId) flatMap { bySourceIds => val item = bySourceIds.floorEntry(mappingFunction(id)) if (item != null) { Some(item.getValue) @@ -157,12 +159,15 @@ class NameServer[T]( } def findForwardings(tableId: Int) = { + import scala.collection.JavaConversions._ + if(forwardings == null) throw new NameserverUninitialized - forwardings.get(tableId).flatMap { bySourceIds => - val shards = bySourceIds.values.toArray(Array[ShardInfo]()).map { shardInfo => + forwardings.get(tableId) map { bySourceIds => + val shards = bySourceIds.values map { shardInfo => findShardById(shardInfo.id) } - Some(shards) + + shards.toList } getOrElse { throw new NonExistentShard("No shards for tableId: %s".format(tableId)) } @@ -175,7 +180,7 @@ class NameServer[T]( @throws(classOf[shards.ShardException]) def getRootShardIds(id: ShardId): Set[ShardId] = { - val ids = nameServerShard.listUpwardLinks(id) + val ids = listUpwardLinks(id) val set: Set[ShardId] = if (ids.isEmpty) Set(id) else Set() // type needed to avoid inferring to Collection[ShardId] set ++ ids.flatMap((i) => getRootShardIds(i.upId).toList) } @@ -184,39 +189,152 @@ class NameServer[T]( ids.map(getRootShardIds).reduceLeft((s1, s2) => s1.filter(s2.contains)).toSeq.headOption } - @throws(classOf[shards.ShardException]) def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = nameServerShard.createShard(shardInfo, repository) - @throws(classOf[shards.ShardException]) def getShard(id: ShardId) = nameServerShard.getShard(id) - @throws(classOf[shards.ShardException]) def deleteShard(id: ShardId) = nameServerShard.deleteShard(id) - @throws(classOf[shards.ShardException]) def addLink(upId: ShardId, downId: ShardId, weight: Int) = nameServerShard.addLink(upId, downId, weight) - @throws(classOf[shards.ShardException]) def removeLink(upId: ShardId, downId: ShardId) = nameServerShard.removeLink(upId, downId) - @throws(classOf[shards.ShardException]) def listUpwardLinks(id: ShardId) = nameServerShard.listUpwardLinks(id) - @throws(classOf[shards.ShardException]) def listDownwardLinks(id: ShardId) = nameServerShard.listDownwardLinks(id) - @throws(classOf[shards.ShardException]) def listLinks() = nameServerShard.listLinks() - @throws(classOf[shards.ShardException]) def markShardBusy(id: ShardId, busy: Busy.Value) = nameServerShard.markShardBusy(id, busy) - @throws(classOf[shards.ShardException]) def setForwarding(forwarding: Forwarding) = nameServerShard.setForwarding(forwarding) - @throws(classOf[shards.ShardException]) def replaceForwarding(oldId: ShardId, newId: ShardId) = nameServerShard.replaceForwarding(oldId, newId) - @throws(classOf[shards.ShardException]) def getForwarding(tableId: Int, baseId: Long) = nameServerShard.getForwarding(tableId, baseId) - @throws(classOf[shards.ShardException]) def getForwardingForShard(id: ShardId) = nameServerShard.getForwardingForShard(id) - @throws(classOf[shards.ShardException]) def getForwardings() = nameServerShard.getForwardings() - @throws(classOf[shards.ShardException]) def shardsForHostname(hostname: String) = nameServerShard.shardsForHostname(hostname) - @throws(classOf[shards.ShardException]) def listShards() = nameServerShard.listShards() - @throws(classOf[shards.ShardException]) def getBusyShards() = nameServerShard.getBusyShards() - @throws(classOf[shards.ShardException]) def rebuildSchema() = nameServerShard.rebuildSchema() - @throws(classOf[shards.ShardException]) def removeForwarding(f: Forwarding) = nameServerShard.removeForwarding(f) - @throws(classOf[shards.ShardException]) def listHostnames() = nameServerShard.listHostnames() - @throws(classOf[shards.ShardException]) def listTables() = nameServerShard.listTables() + @throws(classOf[shards.ShardException]) + def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = { + nameServerShard.writeOperation(_.createShard(shardInfo, repository)) + } + + @throws(classOf[shards.ShardException]) + def getShard(id: ShardId) = { + nameServerShard.readOperation(_.getShard(id)) + } + + @throws(classOf[shards.ShardException]) + def deleteShard(id: ShardId) = { + nameServerShard.writeOperation(_.deleteShard(id)) + } + + @throws(classOf[shards.ShardException]) + def addLink(upId: ShardId, downId: ShardId, weight: Int) = { + nameServerShard.writeOperation(_.addLink(upId, downId, weight)) + } + + @throws(classOf[shards.ShardException]) + def removeLink(upId: ShardId, downId: ShardId) = { + nameServerShard.writeOperation(_.removeLink(upId, downId)) + } + + @throws(classOf[shards.ShardException]) + def listUpwardLinks(id: ShardId) = { + nameServerShard.readOperation(_.listUpwardLinks(id)) + } + + @throws(classOf[shards.ShardException]) + def listDownwardLinks(id: ShardId) = { + nameServerShard.readOperation(_.listDownwardLinks(id)) + } + + @throws(classOf[shards.ShardException]) + def listLinks() = { + nameServerShard.readOperation(_.listLinks()) + } + + @throws(classOf[shards.ShardException]) + def markShardBusy(id: ShardId, busy: Busy.Value) = { + nameServerShard.writeOperation(_.markShardBusy(id, busy)) + } + + @throws(classOf[shards.ShardException]) + def setForwarding(forwarding: Forwarding) = { + nameServerShard.writeOperation(_.setForwarding(forwarding)) + } + + @throws(classOf[shards.ShardException]) + def replaceForwarding(oldId: ShardId, newId: ShardId) = { + nameServerShard.writeOperation(_.replaceForwarding(oldId, newId)) + } + + @throws(classOf[shards.ShardException]) + def getForwarding(tableId: Int, baseId: Long) = { + nameServerShard.readOperation(_.getForwarding(tableId, baseId)) + } + + @throws(classOf[shards.ShardException]) + def getForwardingForShard(id: ShardId) = { + nameServerShard.readOperation(_.getForwardingForShard(id)) + } + + @throws(classOf[shards.ShardException]) + def getForwardings() = { + nameServerShard.readOperation(_.getForwardings()) + } + + @throws(classOf[shards.ShardException]) + def shardsForHostname(hostname: String) = { + nameServerShard.readOperation(_.shardsForHostname(hostname)) + } + + @throws(classOf[shards.ShardException]) + def listShards() = { + nameServerShard.readOperation(_.listShards()) + } + + @throws(classOf[shards.ShardException]) + def getBusyShards() = { + nameServerShard.readOperation(_.getBusyShards()) + } + + @throws(classOf[shards.ShardException]) + def rebuildSchema() = { + nameServerShard.writeOperation(_.rebuildSchema()) + } + + @throws(classOf[shards.ShardException]) + def removeForwarding(f: Forwarding) = { + nameServerShard.writeOperation(_.removeForwarding(f)) + } + + @throws(classOf[shards.ShardException]) + def listHostnames() = { + nameServerShard.readOperation(_.listHostnames()) + } + + @throws(classOf[shards.ShardException]) + def listTables() = { + nameServerShard.readOperation(_.listTables()) + } // Remote Host Management - @throws(classOf[shards.ShardException]) def addRemoteHost(h: Host) = nameServerShard.addRemoteHost(h) - @throws(classOf[shards.ShardException]) def removeRemoteHost(h: String, p: Int) = nameServerShard.removeRemoteHost(h, p) - @throws(classOf[shards.ShardException]) def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = nameServerShard.setRemoteHostStatus(h, p, s) - @throws(classOf[shards.ShardException]) def setRemoteClusterStatus(c: String, s: HostStatus.Value) = nameServerShard.setRemoteClusterStatus(c, s) + @throws(classOf[shards.ShardException]) + def addRemoteHost(h: Host) = { + nameServerShard.writeOperation(_.addRemoteHost(h)) + } + + @throws(classOf[shards.ShardException]) + def removeRemoteHost(h: String, p: Int) = { + nameServerShard.writeOperation(_.removeRemoteHost(h, p)) + } - @throws(classOf[shards.ShardException]) def getRemoteHost(h: String, p: Int) = nameServerShard.getRemoteHost(h, p) - @throws(classOf[shards.ShardException]) def listRemoteClusters() = nameServerShard.listRemoteClusters() - @throws(classOf[shards.ShardException]) def listRemoteHosts() = nameServerShard.listRemoteHosts() - @throws(classOf[shards.ShardException]) def listRemoteHostsInCluster(c: String) = nameServerShard.listRemoteHostsInCluster(c) + @throws(classOf[shards.ShardException]) + def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = { + nameServerShard.writeOperation(_.setRemoteHostStatus(h, p, s)) + } + @throws(classOf[shards.ShardException]) + def setRemoteClusterStatus(c: String, s: HostStatus.Value) = { + nameServerShard.writeOperation(_.setRemoteClusterStatus(c, s)) + } + + + @throws(classOf[shards.ShardException]) + def getRemoteHost(h: String, p: Int) = { + nameServerShard.readOperation(_.getRemoteHost(h, p)) + } + + @throws(classOf[shards.ShardException]) + def listRemoteClusters() = { + nameServerShard.readOperation(_.listRemoteClusters()) + } + + @throws(classOf[shards.ShardException]) + def listRemoteHosts() = { + nameServerShard.readOperation(_.listRemoteHosts()) + } + + @throws(classOf[shards.ShardException]) + def listRemoteHostsInCluster(c: String) = { + nameServerShard.readOperation(_.listRemoteHostsInCluster(c)) + } } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala b/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala deleted file mode 100644 index 7a4da80f..00000000 --- a/src/main/scala/com/twitter/gizzard/nameserver/ReadWriteShardAdapter.scala +++ /dev/null @@ -1,46 +0,0 @@ -package com.twitter.gizzard -package nameserver - -import shards.{RoutingNode, ShardId, ShardInfo, Busy} - -class ReadWriteShardAdapter(node: RoutingNode[Shard]) extends Shard { - def getBusyShards() = node.readOperation(_.getBusyShards()) - def getForwarding(tableId: Int, baseId: Long) = node.readOperation(_.getForwarding(tableId, baseId)) - def getForwardingForShard(id: ShardId) = node.readOperation(_.getForwardingForShard(id)) - def getForwardings() = node.readOperation(_.getForwardings()) - def getForwardingsForTableIds(tableIds: Seq[Int]): Seq[Forwarding] = node.readOperation(_.getForwardingsForTableIds(tableIds)) - def getShard(id: ShardId) = node.readOperation(_.getShard(id)) - def listUpwardLinks(id: ShardId) = node.readOperation(_.listUpwardLinks(id)) - def listDownwardLinks(id: ShardId) = node.readOperation(_.listDownwardLinks(id)) - def listLinks() = node.readOperation(_.listLinks()) - def listShards() = node.readOperation(_.listShards()) - def shardsForHostname(hostname: String) = node.readOperation(_.shardsForHostname(hostname)) - def listHostnames() = node.readOperation(_.listHostnames()) - def listTables() = node.readOperation(_.listTables()) - - def currentState() = node.readOperation(_.currentState()) - - def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = node.writeOperation(_.createShard(shardInfo, repository)) - def deleteShard(id: ShardId) = node.writeOperation(_.deleteShard(id)) - def markShardBusy(id: ShardId, busy: Busy.Value) = node.writeOperation(_.markShardBusy(id, busy)) - def addLink(upId: ShardId, downId: ShardId, weight: Int) = node.writeOperation(_.addLink(upId, downId, weight)) - def removeLink(upId: ShardId, downId: ShardId) = node.writeOperation(_.removeLink(upId, downId)) - def replaceForwarding(oldId: ShardId, newId: ShardId) = node.writeOperation(_.replaceForwarding(oldId, newId)) - def setForwarding(forwarding: Forwarding) = node.writeOperation(_.setForwarding(forwarding)) - def removeForwarding(forwarding: Forwarding) = node.writeOperation(_.removeForwarding(forwarding)) - def reload() = node.writeOperation(_.reload()) - def rebuildSchema() = node.writeOperation(_.rebuildSchema()) - - - // Remote Host Cluster Management - - def addRemoteHost(h: Host) = node.writeOperation(_.addRemoteHost(h)) - def removeRemoteHost(h: String, p: Int) = node.writeOperation(_.removeRemoteHost(h, p)) - def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = node.writeOperation(_.setRemoteHostStatus(h, p, s)) - def setRemoteClusterStatus(c: String, s: HostStatus.Value) = node.writeOperation(_.setRemoteClusterStatus(c, s)) - - def getRemoteHost(h: String, p: Int) = node.readOperation(_.getRemoteHost(h, p)) - def listRemoteClusters() = node.readOperation(_.listRemoteClusters()) - def listRemoteHosts() = node.readOperation(_.listRemoteHosts()) - def listRemoteHostsInCluster(c: String) = node.readOperation(_.listRemoteHostsInCluster(c)) -} diff --git a/src/main/scala/com/twitter/gizzard/proxy/SqlExceptionWrappingProxy.scala b/src/main/scala/com/twitter/gizzard/proxy/SqlExceptionWrappingProxy.scala index 7292fa79..a824def1 100644 --- a/src/main/scala/com/twitter/gizzard/proxy/SqlExceptionWrappingProxy.scala +++ b/src/main/scala/com/twitter/gizzard/proxy/SqlExceptionWrappingProxy.scala @@ -28,8 +28,9 @@ class SqlExceptionWrappingProxy(shardId: ShardId) extends ExceptionHandlingProxy } }) -class SqlExceptionWrappingProxyFactory[S <: shards.Shard](implicit manifest: Manifest[S]) extends ExceptionHandlingProxyFactory[S]({ (shard, e) => - val id = shard.shardInfo.id +class SqlExceptionWrappingProxyFactory[T <: AnyRef : Manifest](id: ShardId) extends ExceptionHandlingProxyFactory[T]({ (shard, e) => + val manifest = implicitly[Manifest[T]] + e match { case e: SqlQueryTimeoutException => throw new shards.ShardTimeoutException(e.timeout, id, e) diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala index ace5c493..eaf142f1 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala @@ -58,6 +58,8 @@ extends RoutingNode[T] { class LeafRoutingNode[T](val shardInfo: ShardInfo, val weight: Int, shard: T) extends RoutingNode[T] { + def this(weight: Int, shard: T) = this(new ShardInfo(new ShardId("", ""), "", "", "", Busy.Normal), weight, shard) + val children: Seq[RoutingNode[T]] = Seq() def readAllOperation[A](f: T => A) = Seq(try { Right(f(shard)) } catch { case e => Left(e) }) diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShardAdapter.scala b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShardAdapter.scala deleted file mode 100644 index 6fba16f0..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShardAdapter.scala +++ /dev/null @@ -1,8 +0,0 @@ -package com.twitter.gizzard.shards - - -abstract class ReadWriteShardAdapter(shard: Shard) extends Shard { - def weight = shard.weight - def children = shard.children - def shardInfo = shard.shardInfo -} \ No newline at end of file diff --git a/src/main/scala/com/twitter/gizzard/shards/Shard.scala b/src/main/scala/com/twitter/gizzard/shards/Shard.scala deleted file mode 100644 index 7040d9af..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/Shard.scala +++ /dev/null @@ -1,8 +0,0 @@ -package com.twitter.gizzard.shards - - -trait Shard { - def shardInfo: ShardInfo - def weight: Int - def children: Seq[Shard] -} diff --git a/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala b/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala index 188e89b3..07884f79 100644 --- a/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala +++ b/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala @@ -6,11 +6,11 @@ import org.specs.mock.{ClassMocker, JMocker} import shards.{ShardException,ShardInfo} class NestableShardFactory extends shards.ShardFactory[Shard] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[Shard]) = new NestableShard(shardInfo, weight, children) + def instantiate(shardInfo: ShardInfo) = new NestableShard(shardInfo) def materialize(shardInfo: ShardInfo) = () } -class NestableShard(val shardInfo: shards.ShardInfo, val weight:Int, val children: Seq[fake.Shard]) extends Shard { +class NestableShard(val shardInfo: shards.ShardInfo) extends Shard { val map = new mutable.HashMap[String, String] def get(key: String) = { diff --git a/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala b/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala index 699ed02b..bbf4bdd7 100644 --- a/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala +++ b/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala @@ -1,10 +1,9 @@ package com.twitter.gizzard package fake -class ReadWriteShardAdapter(shard: shards.ReadWriteShard[Shard]) - extends shards.ReadWriteShardAdapter(shard) with Shard { +import com.twitter.gizzard.shards.RoutingNode - def getAll(k: String) = shard.readAllOperation(_.get(k)) - def get(k: String) = shard.readOperation(_.get(k)) - def put(k: String, v: String) = shard.writeOperation(_.put(k, v)) +class ReadWriteShardAdapter(node: RoutingNode[Shard]) extends Shard { + def get(k: String) = node.readOperation(_.get(k)) + def put(k: String, v: String) = node.writeOperation(_.put(k, v)) } diff --git a/src/test/scala/com/twitter/gizzard/fake/Shard.scala b/src/test/scala/com/twitter/gizzard/fake/Shard.scala index 597c478b..c8b783ea 100644 --- a/src/test/scala/com/twitter/gizzard/fake/Shard.scala +++ b/src/test/scala/com/twitter/gizzard/fake/Shard.scala @@ -1,7 +1,7 @@ package com.twitter.gizzard package fake -trait Shard extends shards.Shard { +trait Shard extends { @throws(classOf[Throwable]) def get(k: String): Option[String] @throws(classOf[Throwable]) def put(k: String, v: String):String } diff --git a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala index 771430b8..63f7c713 100644 --- a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala +++ b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala @@ -8,9 +8,9 @@ import com.twitter.querulous.config.Connection import com.twitter.querulous.query.SqlQueryTimeoutException import com.twitter.gizzard -import nameserver.NameServer -import shards.{ShardId, ShardInfo, ShardException, ShardTimeoutException} -import scheduler.{JobScheduler, JsonJob, CopyJob, CopyJobParser, CopyJobFactory, JsonJobParser, PrioritizingJobScheduler} +import com.twitter.gizzard.nameserver.NameServer +import com.twitter.gizzard.shards.{RoutingNode, ShardId, ShardInfo, ShardException, ShardTimeoutException} +import com.twitter.gizzard.scheduler.{JobScheduler, JsonJob, CopyJob, CopyJobParser, CopyJobFactory, JsonJobParser, PrioritizingJobScheduler} package object config { import com.twitter.gizzard.config._ @@ -90,21 +90,21 @@ object Priority extends Enumeration { class TestServer(conf: config.TestServer) extends GizzardServer[TestShard](conf) { // shard/nameserver/scheduler wiring - - val readWriteShardAdapter = new TestReadWriteAdapter(_) val jobPriorities = List(Priority.High.id, Priority.Low.id) val copyPriority = Priority.Low.id val copyFactory = new TestCopyFactory(nameServer, jobScheduler(Priority.Low.id)) shardRepo += ("TestShard" -> new SqlShardFactory(conf.queryEvaluator(), conf.databaseConnection)) - jobCodec += ("Put".r -> new PutParser(nameServer.findCurrentForwarding(0, _))) + def findForwarding(id: Long) = new TestShardAdapter(nameServer.findCurrentForwarding(0, id)) + + jobCodec += ("Put".r -> new PutParser(findForwarding)) jobCodec += ("Copy".r -> new TestCopyParser(nameServer, jobScheduler(Priority.Low.id))) // service listener - val testService = new TestServerIFace(nameServer.findCurrentForwarding(0, _), jobScheduler) + val testService = new TestServerIFace(findForwarding, jobScheduler) lazy val testThriftServer = { val processor = new thrift.TestServer.Processor(testService) @@ -142,25 +142,23 @@ extends thrift.TestServer.Iface { // Shard Definitions -trait TestShard extends shards.Shard { +trait TestShard { def put(key: Int, value: String): Unit def putAll(kvs: Seq[(Int, String)]): Unit def get(key: Int): Option[(Int, String, Int)] def getAll(key: Int, count: Int): Seq[(Int, String, Int)] } -class TestReadWriteAdapter(s: shards.ReadWriteShard[TestShard]) -extends shards.ReadWriteShardAdapter(s) with TestShard { +class TestShardAdapter(s: shards.RoutingNode[TestShard]) extends TestShard { def put(k: Int, v: String) = s.writeOperation(_.put(k,v)) def putAll(kvs: Seq[(Int,String)]) = s.writeOperation(_.putAll(kvs)) def get(k: Int) = s.readOperation(_.get(k)) def getAll(k:Int, c: Int) = s.readOperation(_.getAll(k,c)) } -class SqlShardFactory(qeFactory: QueryEvaluatorFactory, conn: Connection) -extends shards.ShardFactory[TestShard] { - def instantiate(info: ShardInfo, weight: Int, children: Seq[TestShard]) = - new SqlShard(qeFactory(conn.withHost(info.hostname)), info, weight, children) +class SqlShardFactory(qeFactory: QueryEvaluatorFactory, conn: Connection) extends shards.ShardFactory[TestShard] { + + def instantiate(info: ShardInfo) = new SqlShard(qeFactory(conn.withHost(info.hostname)), info) def materialize(info: ShardInfo) { val ddl = @@ -181,16 +179,12 @@ extends shards.ShardFactory[TestShard] { } } -class SqlShard( - evaluator: QueryEvaluator, - val shardInfo: ShardInfo, - val weight: Int, - val children: Seq[TestShard]) -extends TestShard { +class SqlShard(evaluator: QueryEvaluator, val shardInfo: ShardInfo) extends TestShard { + private val table = shardInfo.tablePrefix - private val putSql = """insert into %s (id, value, count) values (?,?,1) on duplicate key - update value = values(value), count = count+1""".format(table) + private val putSql = """insert into %s (id, value, count) values (?,?,1) on duplicate key + update value = values(value), count = count+1""".format(table) private val getSql = "select * from " + table + " where id = ?" private val getAllSql = "select * from " + table + " where id > ? limit ?" @@ -236,7 +230,8 @@ extends CopyJobParser[TestShard] { class TestCopy(srcId: ShardId, destId: ShardId, cursor: Int, count: Int, ns: NameServer[TestShard], s: JobScheduler) extends CopyJob[TestShard](srcId, destId, count, ns, s) { - def copyPage(src: TestShard, dest: TestShard, count: Int) = { + def copyPage(srcNode: RoutingNode[TestShard], destNode: RoutingNode[TestShard], count: Int) = { + val List(src, dest) = List(srcNode, destNode).map(new TestShardAdapter(_)) val rows = src.getAll(cursor, count).map { case (k,v,c) => (k,v) } if (rows.isEmpty) { diff --git a/src/test/scala/com/twitter/gizzard/nameserver/LoadBalancerSpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/LoadBalancerSpec.scala index 1524da91..800b9ec9 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/LoadBalancerSpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/LoadBalancerSpec.scala @@ -5,6 +5,8 @@ import org.specs.Specification import org.specs.mock.{ClassMocker, JMocker} import scala.util.Random +import com.twitter.gizzard.shards.RoutingNode + // FIXME: these tests kinda suck in theory. Ideally, we'd test based on // a fuzzy expectation of the distribution of responses. @@ -13,9 +15,9 @@ import scala.util.Random object LoadBalancerSpec extends ConfiguredSpecification with JMocker with ClassMocker { "LoadBalancer" should { val random = new Random(0) - val shard1 = mock[shards.Shard] - val shard2 = mock[shards.Shard] - val shard3 = mock[shards.Shard] + val shard1 = mock[RoutingNode[AnyRef]] + val shard2 = mock[RoutingNode[AnyRef]] + val shard3 = mock[RoutingNode[AnyRef]] "with a zero weight" in { expect { @@ -42,9 +44,9 @@ object LoadBalancerSpec extends ConfiguredSpecification with JMocker with ClassM object FailingOverLoadBalancerSpec extends ConfiguredSpecification with JMocker with ClassMocker { "FailingOverLoadBalancer" should { val random = new Random(0) - val shard1 = mock[shards.Shard] - val shard2 = mock[shards.Shard] - val shard3 = mock[shards.Shard] + val shard1 = mock[RoutingNode[AnyRef]] + val shard2 = mock[RoutingNode[AnyRef]] + val shard3 = mock[RoutingNode[AnyRef]] "returns 1 online followed by randomly ordered offlines followed by rest of onlines" in { expect { diff --git a/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala index 89b9a430..749eef75 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala @@ -5,16 +5,19 @@ import org.specs.Specification import org.specs.mock.{ClassMocker, JMocker} import com.twitter.gizzard +import com.twitter.gizzard.shards.LeafRoutingNode object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMocker { "NameServer" should { val SQL_SHARD = "com.example.SqlShard" - val nameServerShard = mock[Shard] - var shardRepository = mock[ShardRepository[shards.Shard]] + val nameServerShard = mock[nameserver.Shard] + val routingNode = new LeafRoutingNode(1, nameServerShard) + + var shardRepository = mock[ShardRepository[AnyRef]] val mappingFunction = (n: Long) => n - var nameServer: NameServer[shards.Shard] = null + var nameServer: NameServer[AnyRef] = null val shardInfos = (1 until 5).toList.map { id => new shards.ShardInfo(shards.ShardId("localhost", id.toString), SQL_SHARD, "a", "b", shards.Busy.Normal) @@ -29,7 +32,8 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc new Host("host2", 7777, "c1", HostStatus.Normal), new Host("host3", 7777, "c2", HostStatus.Normal)) - var shard = mock[shards.Shard] + val shard = mock[AnyRef] + val node = new LeafRoutingNode(1, shard) doBefore { expect { @@ -38,8 +42,8 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc one(nameServerShard).currentState() willReturn Seq(nameServerState) } - nameServer = new NameServer[gizzard.shards.Shard]( - nameServerShard, + nameServer = new NameServer[AnyRef]( + routingNode, shardRepository, NullJobRelayFactory, mappingFunction) @@ -71,30 +75,30 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc "find current forwarding" in { expect { - one(shardRepository).find(shardInfos(1), 1, List()) willReturn shard + one(shardRepository).find(shardInfos(1), 1, List()) willReturn node } - nameServer.findCurrentForwarding(1, 2) mustEqual shard + nameServer.findCurrentForwarding(1, 2) mustEqual node } "find forwardings" in { expect { - one(shardRepository).find(shardInfos(3), 1, List()) willReturn shard - one(shardRepository).find(shardInfos(0), 1, List()) willReturn shard - one(shardRepository).find(shardInfos(1), 1, List()) willReturn shard - one(shardRepository).find(shardInfos(2), 1, List(shard)) willReturn shard + one(shardRepository).find(shardInfos(3), 1, List()) willReturn node + one(shardRepository).find(shardInfos(0), 1, List()) willReturn node + one(shardRepository).find(shardInfos(1), 1, List()) willReturn node + one(shardRepository).find(shardInfos(2), 1, List(node)) willReturn node } - nameServer.findForwardings(1) mustEqual List(shard, shard, shard) + nameServer.findForwardings(1) mustEqual List(node, node, node) } "find shard by id" in { expect { - one(shardRepository).find(shardInfos(3), 1, List()) willReturn shard - one(shardRepository).find(shardInfos(2), 1, List(shard)) willReturn shard + one(shardRepository).find(shardInfos(3), 1, List()) willReturn node + one(shardRepository).find(shardInfos(2), 1, List(node)) willReturn node } - nameServer.findShardById(shardInfos(2).id) mustEqual shard + nameServer.findShardById(shardInfos(2).id) mustEqual node } "find shard by id with a shard not attached to a forwarding" in { @@ -103,10 +107,10 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc expect { one(nameServerShard).getShard(floatingShard.id) willReturn floatingShard one(nameServerShard).listDownwardLinks(floatingShard.id) willReturn List[shards.LinkInfo]() - one(shardRepository).find(floatingShard, 1, List()) willReturn shard + one(shardRepository).find(floatingShard, 1, List()) willReturn node } - nameServer.findShardById(floatingShard.id) mustEqual shard + nameServer.findShardById(floatingShard.id) mustEqual node } "create shard" in { diff --git a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala index 8b6c2dbe..772ea924 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala @@ -1,41 +1,42 @@ package com.twitter.gizzard package nameserver -import com.twitter.util.TimeConversions._ +import com.twitter.conversions.time._ import org.specs.Specification import org.specs.mock.{ClassMocker, JMocker} +import com.twitter.gizzard.shards.RoutingNode + object ShardRepositorySpec extends ConfiguredSpecification with JMocker with ClassMocker { "BasicShardRepository" should { val future = mock[Future] - val shard = mock[shards.Shard] - val constructor = { (shard: shards.ReadWriteShard[shards.Shard]) => shard } - val repository = new BasicShardRepository(constructor, Some(future)) + val shard = mock[RoutingNode[AnyRef]] + val repository = new BasicShardRepository(Some(future)) "find a read-only shard" in { - repository.factory("ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[shards.Shard]] - repository.factory("com.twitter.gizzard.shards.ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[shards.Shard]] + repository.factory("ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[AnyRef]] + repository.factory("com.twitter.gizzard.shards.ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[AnyRef]] repository.factory("com.example.bogis.ReadOnlyShard") must throwA[NoSuchElementException] } "find a write-only shard" in { - repository.factory("WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[shards.Shard]] - repository.factory("com.twitter.gizzard.shards.WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[shards.Shard]] + repository.factory("WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[AnyRef]] + repository.factory("com.twitter.gizzard.shards.WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[AnyRef]] repository.factory("com.example.bogis.WriteOnlyShard") must throwA[NoSuchElementException] } "find a blocked shard" in { - repository.factory("BlockedShard") must haveClass[shards.BlockedShardFactory[shards.Shard]] - repository.factory("com.twitter.gizzard.shards.BlockedShard") must haveClass[shards.BlockedShardFactory[shards.Shard]] + repository.factory("BlockedShard") must haveClass[shards.BlockedShardFactory[AnyRef]] + repository.factory("com.twitter.gizzard.shards.BlockedShard") must haveClass[shards.BlockedShardFactory[AnyRef]] repository.factory("com.example.bogis.BlockedShard") must throwA[NoSuchElementException] } "find a replicating shard" in { - repository.factory("ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[shards.Shard]] - repository.factory("com.twitter.gizzard.shards.ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[shards.Shard]] + repository.factory("ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[AnyRef]] + repository.factory("com.twitter.gizzard.shards.ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[AnyRef]] repository.factory("com.example.bogis.ReplicatingShard") must throwA[NoSuchElementException] } "find a failing over shard" in { - repository.factory("FailingOverShard") must haveClass[shards.FailingOverShardFactory[shards.Shard]] - repository.factory("com.twitter.gizzard.shards.FailingOverShard") must haveClass[shards.FailingOverShardFactory[shards.Shard]] + repository.factory("FailingOverShard") must haveClass[shards.FailingOverShardFactory[AnyRef]] + repository.factory("com.twitter.gizzard.shards.FailingOverShard") must haveClass[shards.FailingOverShardFactory[AnyRef]] repository.factory("com.example.bogis.FailingOverShard") must throwA[NoSuchElementException] } } diff --git a/src/test/scala/com/twitter/gizzard/nameserver/SqlShardSpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/SqlShardSpec.scala index aede3743..f01e7781 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/SqlShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/SqlShardSpec.scala @@ -19,11 +19,10 @@ class SqlShardSpec extends ConfiguredSpecification with JMocker with ClassMocker val SQL_SHARD = "com.example.SqlShard" var nameServer: nameserver.SqlShard = null - var shardRepository: ShardRepository[Shard] = null - val adapter = { (shard:shards.ReadWriteShard[fake.Shard]) => new fake.ReadWriteShardAdapter(shard) } + var shardRepository: ShardRepository[fake.Shard] = null val future = new Future("Future!", 1, 1, 1.second, 1.second) - val repo = new BasicShardRepository[fake.Shard](adapter, Some(future)) + val repo = new BasicShardRepository[fake.Shard](Some(future)) repo += ("com.twitter.gizzard.fake.NestableShard" -> new fake.NestableShardFactory()) val forwardShardInfo = new ShardInfo(SQL_SHARD, "forward_table", "localhost") @@ -33,17 +32,7 @@ class SqlShardSpec extends ConfiguredSpecification with JMocker with ClassMocker nameServer = new SqlShard(queryEvaluator) nameServer.rebuildSchema() reset(config.nameServer) - shardRepository = mock[ShardRepository[Shard]] - } - - "be wrappable while replicating" in { - val nameServerShards = Seq(nameServer) - val info = new shards.ShardInfo("com.twitter.gizzard.nameserver.Replicatingnameserver.NameServer", "", "") - val replicationFuture = new Future("ReplicationFuture", 1, 1, 1.second, 1.second) - val shard: shards.ReadWriteShard[nameserver.Shard] = - new shards.ReplicatingShard(info, 0, nameServerShards, new nameserver.LoadBalancer(nameServerShards), Some(replicationFuture)) - val adapted = new nameserver.ReadWriteShardAdapter(shard) - 1 mustEqual 1 + shardRepository = mock[ShardRepository[fake.Shard]] } "be able to dump nameserver structure" in { diff --git a/src/test/scala/com/twitter/gizzard/proxy/ExceptionProxySpec.scala b/src/test/scala/com/twitter/gizzard/proxy/ExceptionProxySpec.scala index 9a1adf88..afa5c6da 100644 --- a/src/test/scala/com/twitter/gizzard/proxy/ExceptionProxySpec.scala +++ b/src/test/scala/com/twitter/gizzard/proxy/ExceptionProxySpec.scala @@ -9,15 +9,14 @@ import org.specs.mock.JMocker object SqlExceptionWrappingProxySpec extends ConfiguredSpecification with JMocker { "SqlExceptionWrappingProxy" should { val shard = mock[fake.Shard] - val proxyFactory = new SqlExceptionWrappingProxyFactory[fake.Shard] - val shardProxy = proxyFactory(shard) val shardInfo = ShardInfo(ShardId("test", "shard"), "fake.shard", "blah", "blah", Busy.Normal) + val proxyFactory = new SqlExceptionWrappingProxyFactory[fake.Shard](shardInfo.id) + val shardProxy = proxyFactory(shard) val sqlException = new SQLException("huh!") "wrap exceptions" in { expect { one(shard).get("blah") willThrow sqlException - one(shard).shardInfo willReturn shardInfo } shardProxy.get("blah") must throwA(new ShardException(sqlException.toString, sqlException)) diff --git a/src/test/scala/com/twitter/gizzard/scheduler_new/CopyJobSpec.scala b/src/test/scala/com/twitter/gizzard/scheduler_new/CopyJobSpec.scala index 78712f77..2d61cd2c 100644 --- a/src/test/scala/com/twitter/gizzard/scheduler_new/CopyJobSpec.scala +++ b/src/test/scala/com/twitter/gizzard/scheduler_new/CopyJobSpec.scala @@ -1,18 +1,25 @@ package com.twitter.gizzard package scheduler -import com.twitter.util.TimeConversions._ +import com.twitter.conversions.time._ import org.specs.Specification import org.specs.mock.{ClassMocker, JMocker} +import com.twitter.gizzard.shards.RoutingNode + +class FakeCopy( + val sourceShardId: shards.ShardId, + val destinationShardId: shards.ShardId, + count: Int, + nameServer: nameserver.NameServer[AnyRef], + scheduler: JobScheduler, + nextCopy: => Option[FakeCopy]) +extends CopyJob[AnyRef](sourceShardId, destinationShardId, count, nameServer, scheduler) { -class FakeCopy(val sourceShardId: shards.ShardId, val destinationShardId: shards.ShardId, count: Int, - nameServer: nameserver.NameServer[shards.Shard], scheduler: JobScheduler)(nextCopy: => Option[FakeCopy]) - extends CopyJob[shards.Shard](sourceShardId, destinationShardId, count, nameServer, scheduler) { def serialize = Map("cursor" -> 1) @throws(classOf[Exception]) - def copyPage(sourceShard: shards.Shard, destinationShard: shards.Shard, count: Int) = nextCopy + def copyPage(sourceShard: RoutingNode[AnyRef], destinationShard: RoutingNode[AnyRef], count: Int) = nextCopy override def equals(that: Any) = that match { case that: FakeCopy => @@ -29,11 +36,11 @@ object CopyJobSpec extends ConfiguredSpecification with JMocker with ClassMocker val destinationShardInfo = shards.ShardInfo(destinationShardId, "FakeShard", "", "", shards.Busy.Normal) val count = CopyJob.MIN_COPY + 1 val nextCopy = mock[FakeCopy] - val nameServer = mock[nameserver.NameServer[shards.Shard]] + val nameServer = mock[nameserver.NameServer[AnyRef]] val jobScheduler = mock[JobScheduler] - val makeCopy = new FakeCopy(sourceShardId, destinationShardId, count, nameServer, jobScheduler)(_) - val shard1 = mock[shards.Shard] - val shard2 = mock[shards.Shard] + def makeCopy(next: => Option[FakeCopy]) = new FakeCopy(sourceShardId, destinationShardId, count, nameServer, jobScheduler, next) + val shard1 = mock[RoutingNode[AnyRef]] + val shard2 = mock[RoutingNode[AnyRef]] "toMap" in { val copy = makeCopy(Some(nextCopy)) @@ -133,7 +140,7 @@ object CopyJobSpec extends ConfiguredSpecification with JMocker with ClassMocker "after too many retries" in { val count = CopyJob.MIN_COPY - 1 - val copy = new FakeCopy(sourceShardId, destinationShardId, count, nameServer, jobScheduler)(throw new shards.ShardTimeoutException(100.milliseconds, sourceShardId)) + val copy = new FakeCopy(sourceShardId, destinationShardId, count, nameServer, jobScheduler, throw new shards.ShardTimeoutException(100.milliseconds, sourceShardId)) expect { one(nameServer).getShard(destinationShardId) willReturn destinationShardInfo diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 2d1b57ab..1d4bc98c 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -8,43 +8,51 @@ import org.specs.mock.JMocker import com.twitter.gizzard.nameserver.LoadBalancer import com.twitter.ostrich.W3CReporter + object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { "ReplicatingShard" should { val shardId = ShardId("fake", "shard") val shard1 = mock[fake.Shard] val shard2 = mock[fake.Shard] val shard3 = mock[fake.Shard] + val List(node1, node2, node3) = List(shard1, shard2, shard3).map(new LeafRoutingNode(1, _)) + val future = new Future("Future!", 1, 1, 1.second, 1.second) - val shards = List(shard1, shard2) - val loadBalancer = () => shards + val shards = List(node1, node2) + + def loadBalancer() = shards val replicatingShardInfo = new ShardInfo("", "replicating_shard", "hostname") - var replicatingShard = new fake.ReadWriteShardAdapter(new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, Some(future))) + var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, Some(future)) "read failover" in { "when shard1 throws an exception" in { val shard1Info = new ShardInfo("", "table_prefix", "hostname") val exception = new ShardException("o noes") expect { - one(shard1).shardInfo.willReturn(shard1Info) one(shard1).get("name").willThrow(exception) then one(shard2).get("name").willReturn(Some("bob")) } - replicatingShard.get("name") mustEqual Some("bob") + replicatingShard.readOperation(_.get("name")) mustEqual Some("bob") } "when all shards throw an exception" in { val shard1Info = new ShardInfo("", "table_prefix", "hostname") val exception = new ShardException("o noes") expect { - one(shard1).shardInfo willReturn shard1Info - one(shard2).shardInfo willReturn shard1Info one(shard1).get("name") willThrow exception one(shard2).get("name") willThrow exception } - replicatingShard.get("name") must throwA[ShardException] + replicatingShard.readOperation(_.get("name")) must throwA[ShardException] } } + "reads happen to shards in order" in { + expect { + one(shard1).get("name").willReturn(Some("ted")) + } + replicatingShard.readOperation(_.get("name")) mustEqual Some("ted") + } + "read all shards" in { "when all succeed" in { expect { @@ -52,7 +60,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willReturn Some("bob") } - replicatingShard.getAll("name") must haveTheSameElementsAs(List(Right(Some("joe")), Right(Some("bob")))) + replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Right(Some("joe")), Right(Some("bob")))) } "when one fails" in { @@ -63,7 +71,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willReturn Some("bob") } - replicatingShard.getAll("name") must haveTheSameElementsAs(List(Left(ex), Right(Some("bob")))) + replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Left(ex), Right(Some("bob")))) } "when all fail" in { @@ -75,7 +83,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willThrow ex2 } - replicatingShard.getAll("name") must haveTheSameElementsAs(List(Left(ex1), Left(ex2))) + replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Left(ex1), Left(ex2))) } } @@ -86,7 +94,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") one(shard2).put("name", "alice") } - replicatingShard.put("name", "alice") + replicatingShard.writeOperation(_.put("name", "alice")) } "when the first one fails" in { @@ -94,7 +102,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") willThrow new ShardException("o noes") one(shard2).put("name", "alice") } - replicatingShard.put("name", "alice") must throwA[Exception] + replicatingShard.writeOperation(_.put("name", "alice")) must throwA[Exception] } "when one replica is black holed" in { @@ -102,7 +110,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") willThrow new ShardBlackHoleException(shardId) one(shard2).put("name", "alice") } - replicatingShard.put("name", "alice") + replicatingShard.writeOperation(_.put("name", "alice")) } "when all replicas are black holed" in { @@ -110,19 +118,19 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") willThrow new ShardBlackHoleException(shardId) one(shard2).put("name", "alice") willThrow new ShardBlackHoleException(shardId) } - replicatingShard.put("name", "alice") must throwA[ShardBlackHoleException] + replicatingShard.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] } } "in series" in { - var replicatingShard = new fake.ReadWriteShardAdapter(new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, None)) + var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, None) "normal" in { expect { one(shard1).put("name", "carol") one(shard2).put("name", "carol") } - replicatingShard.put("name", "carol") + replicatingShard.writeOperation(_.put("name", "carol")) } "with an exception" in { @@ -130,7 +138,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "carol") willThrow new ShardException("o noes") one(shard2).put("name", "carol") } - replicatingShard.put("name", "carol") must throwA[ShardException] + replicatingShard.writeOperation(_.put("name", "carol")) must throwA[ShardException] } "with a black hole" in { @@ -138,7 +146,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "carol") willThrow new ShardBlackHoleException(shardId) one(shard2).put("name", "carol") } - replicatingShard.put("name", "carol") + replicatingShard.writeOperation(_.put("name", "carol")) } "with all black holes" in { @@ -146,20 +154,13 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "carol") willThrow new ShardBlackHoleException(shardId) one(shard2).put("name", "carol") willThrow new ShardBlackHoleException(shardId) } - replicatingShard.put("name", "carol") must throwA[ShardBlackHoleException] + replicatingShard.writeOperation(_.put("name", "carol")) must throwA[ShardBlackHoleException] } } } - "reads happen to shards in order" in { - expect { - one(shard1).get("name").willReturn(Some("ted")) - } - replicatingShard.get("name") mustEqual Some("ted") - } - "rebuildableFailover" in { - trait EnufShard extends Shard { + trait EnufShard { @throws(classOf[ShardException]) def getPrice: Option[Int] @throws(classOf[ShardException]) def setPrice(price: Int) } @@ -167,7 +168,8 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val shardInfo = new ShardInfo("fake", "fake", "localhost") val mock1 = mock[EnufShard] val mock2 = mock[EnufShard] - val shards = List(mock1, mock2) + val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(1, _)) + val shards = List(node1, node2) val loadBalancer = () => shards val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards, loadBalancer, Some(future)) @@ -183,7 +185,6 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { expect { one(mock1).getPrice willThrow new ShardException("oof!") one(mock2).getPrice willReturn Some(100) - allowing(mock1).shardInfo willReturn shardInfo } shard.rebuildableReadOperation(_.getPrice) { (shard, destShard) => destShard.setPrice(shard.getPrice.get) } mustEqual Some(100) @@ -212,8 +213,6 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { expect { one(mock1).getPrice willThrow new ShardException("oof!") one(mock2).getPrice willThrow new ShardException("oof!") - allowing(mock1).shardInfo willReturn shardInfo - allowing(mock2).shardInfo willReturn shardInfo } shard.rebuildableReadOperation(_.getPrice) { (shard, destShard) => destShard.setPrice(shard.getPrice.get) } must throwA[ShardOfflineException] diff --git a/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala index cdd47732..89ff7346 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala @@ -17,7 +17,7 @@ object ShardsIntegrationSpec extends ConfiguredSpecification with JMocker with C val queryEvaluator = evaluator(config.nameServer) materialize(config.nameServer) - class UserShard(val shardInfo: ShardInfo, val weight: Int, val children: Seq[Shard]) extends Shard { + class UserShard(val shardInfo: ShardInfo) { val data = new mutable.HashMap[Int, String] def setName(id: Int, name: String) { @@ -28,18 +28,16 @@ object ShardsIntegrationSpec extends ConfiguredSpecification with JMocker with C } val factory = new ShardFactory[UserShard] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[UserShard]) = { - new UserShard(shardInfo, weight, children) + def instantiate(shardInfo: ShardInfo) = { + new UserShard(shardInfo) } - def materialize(shardInfo: ShardInfo) { - // nothing. - } + def materialize(shardInfo: ShardInfo) {} } "Shards" should { var shardRepository: ShardRepository[UserShard] = null - var nameServerShard: nameserver.Shard = null + var nameServerShard: RoutingNode[nameserver.Shard] = null var nameServer: NameServer[UserShard] = null var mapping = (a: Long) => a @@ -49,7 +47,7 @@ object ShardsIntegrationSpec extends ConfiguredSpecification with JMocker with C shardRepository += (("com.example.UserShard", factory)) shardRepository += (("com.example.SqlShard", factory)) reset(queryEvaluator) - nameServerShard = new SqlShard(queryEvaluator) + nameServerShard = new LeafRoutingNode(1, new SqlShard(queryEvaluator)) nameServer = new NameServer(nameServerShard, shardRepository, NullJobRelayFactory, mapping) nameServer.reload() diff --git a/src/test/scala/com/twitter/gizzard/thrift/ShardManagerServiceSpec.scala b/src/test/scala/com/twitter/gizzard/thrift/ShardManagerServiceSpec.scala index fec206bb..1f6f649a 100644 --- a/src/test/scala/com/twitter/gizzard/thrift/ShardManagerServiceSpec.scala +++ b/src/test/scala/com/twitter/gizzard/thrift/ShardManagerServiceSpec.scala @@ -7,20 +7,20 @@ import org.specs.Specification import com.twitter.gizzard.thrift.conversions.Sequences._ import com.twitter.gizzard.thrift.conversions.ShardId._ import com.twitter.gizzard.thrift.conversions.ShardInfo._ -import shards.{Busy, Shard} +import shards.{Busy, RoutingNode} import scheduler.{CopyJob, CopyJobFactory, JobScheduler, PrioritizingJobScheduler, JsonJob} object ManagerServiceSpec extends ConfiguredSpecification with JMocker with ClassMocker { - val nameServer = mock[nameserver.NameServer[Shard]] - val copier = mock[CopyJobFactory[Shard]] + val nameServer = mock[nameserver.NameServer[AnyRef]] + val copier = mock[CopyJobFactory[AnyRef]] val scheduler = mock[PrioritizingJobScheduler] val subScheduler = mock[JobScheduler] val copyScheduler = mock[JobScheduler] val manager = new ManagerService(nameServer, copier, scheduler, copyScheduler, null, 0, null) - val shard = mock[Shard] + val shard = mock[RoutingNode[AnyRef]] val thriftShardInfo1 = new thrift.ShardInfo(new thrift.ShardId("hostname", "table_prefix"), "com.example.SqlShard", "INT UNSIGNED", "INT UNSIGNED", Busy.Normal.id) val shardInfo1 = new shards.ShardInfo(new shards.ShardId("hostname", "table_prefix"), @@ -112,7 +112,7 @@ object ManagerServiceSpec extends ConfiguredSpecification with JMocker with Clas "copy_shard" in { val shardId1 = new shards.ShardId("hostname1", "table1") val shardId2 = new shards.ShardId("hostname2", "table2") - val copyJob = mock[CopyJob[Shard]] + val copyJob = mock[CopyJob[AnyRef]] expect { one(copier).apply(shardId1, shardId2) willReturn copyJob From 3063188e0c3b134fc26ef9a0c120ea647250faf2 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 28 Apr 2011 10:09:08 -0700 Subject: [PATCH 03/16] move things around in shards pkg --- .../twitter/gizzard/config/NameServer.scala | 3 +- .../gizzard/shards/BlackHoleShard.scala | 24 ----- .../twitter/gizzard/shards/BlockedShard.scala | 21 ---- .../gizzard/shards/LeafRoutingNode.scala | 32 ++++++ .../gizzard/shards/ReadOnlyShard.scala | 17 ---- .../gizzard/shards/ReadWriteShard.scala | 85 ---------------- .../twitter/gizzard/shards/RoutingNode.scala | 42 ++++++++ .../twitter/gizzard/shards/ShardFactory.scala | 2 +- .../gizzard/shards/WrapperRoutingNodes.scala | 97 +++++++++++++++++++ .../gizzard/shards/WriteOnlyShard.scala | 19 ---- .../gizzard/test/FakeReadWriteShard.scala | 4 +- .../twitter/gizzard/fake/NestableShard.scala | 16 +-- .../gizzard/integration/TestServer.scala | 4 +- .../gizzard/nameserver/NameServerSpec.scala | 4 +- .../gizzard/shards/ReplicatingShardSpec.scala | 4 +- .../shards/ShardsIntegrationSpec.scala | 4 +- 16 files changed, 193 insertions(+), 185 deletions(-) delete mode 100644 src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala create mode 100644 src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala create mode 100644 src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala create mode 100644 src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala delete mode 100644 src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala diff --git a/src/main/scala/com/twitter/gizzard/config/NameServer.scala b/src/main/scala/com/twitter/gizzard/config/NameServer.scala index 47bef347..982a1836 100644 --- a/src/main/scala/com/twitter/gizzard/config/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/config/NameServer.scala @@ -61,8 +61,7 @@ trait NameServer { } def apply[T](shardRepository: nameserver.ShardRepository[T]) = { - val replicaInfo = new shards.ShardInfo("", "", "") - val replicaNodes = replicas map { r => new shards.LeafRoutingNode(replicaInfo, 1, r()) } + val replicaNodes = replicas map { replica => new shards.LeafRoutingNode(replica(), 1) } val shardInfo = new shards.ShardInfo("com.twitter.gizzard.nameserver.ReplicatingShard", "", "") val loadBalancer = new nameserver.LoadBalancer(replicaNodes) diff --git a/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala b/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala deleted file mode 100644 index 88ab90b5..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/BlackHoleShard.scala +++ /dev/null @@ -1,24 +0,0 @@ -package com.twitter.gizzard -package shards - - -class BlackHoleShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new BlackHoleShard(shardInfo, weight, children) - } -} - -/** - * A shard that refuses all read/write traffic in a silent way. - */ -class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends PassThroughNode[T](shardInfo, weight, children) { - - private def exception = throw new ShardBlackHoleException(shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]]() - override def readOperation[A](method: T => A) = throw exception - override def writeOperation[A](method: T => A) = throw exception - - override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception -} diff --git a/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala b/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala deleted file mode 100644 index 97c3415a..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/BlockedShard.scala +++ /dev/null @@ -1,21 +0,0 @@ -package com.twitter.gizzard -package shards - - -class BlockedShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new BlockedShard(shardInfo, weight, children) - } -} - -class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends PassThroughNode[T](shardInfo, weight, children) { - - private def exception = new ShardRejectedOperationException("shard is offline", shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq(Left(exception)) - override def readOperation[A](method: T => A) = throw exception - override def writeOperation[A](method: T => A) = throw exception - - override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception -} diff --git a/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala new file mode 100644 index 00000000..a82a807b --- /dev/null +++ b/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala @@ -0,0 +1,32 @@ +package com.twitter.gizzard.shards + + +class LeafRoutingNode[T](shard: T, val shardInfo: ShardInfo, val weight: Int) extends RoutingNode[T] { + + val children = Nil + + // convenience constructor for manual creation. + def this(shard: T, weight: Int) = this(shard, new ShardInfo("", "", ""), weight) + + def readAllOperation[A](f: T => A) = Seq(try { Right(f(shard)) } catch { case e => Left(e) }) + def readOperation[A](f: T => A) = f(shard) + def writeOperation[A](f: T => A) = f(shard) + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + f(shard, toRebuild) match { + case Some(rv) => Right(rv) + case None => Left(shard :: toRebuild) + } + } +} + +class LeafRoutingNodeFactory[T](shardFactory: ShardFactory[T]) extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + val shard = shardFactory.instantiate(shardInfo, weight) + new LeafRoutingNode(shard, shardInfo, weight) + } + + override def materialize(shardInfo: ShardInfo) { + shardFactory.materialize(shardInfo) + } +} diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala deleted file mode 100644 index 32faa746..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/ReadOnlyShard.scala +++ /dev/null @@ -1,17 +0,0 @@ -package com.twitter.gizzard -package shards - - -class ReadOnlyShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new ReadOnlyShard(shardInfo, weight, children) - } -} - -class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends PassThroughNode[T](shardInfo, weight, children) { - - override def writeOperation[A](method: T => A) = { - throw new ShardRejectedOperationException("shard is read-only", shardInfo.id) - } -} diff --git a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala deleted file mode 100644 index eaf142f1..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/ReadWriteShard.scala +++ /dev/null @@ -1,85 +0,0 @@ -package com.twitter.gizzard.shards - -import java.lang.reflect.UndeclaredThrowableException -import java.util.concurrent.{ExecutionException, TimeoutException} - - -abstract class RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]): RoutingNode[T] - def materialize(shardInfo: ShardInfo) {} -} - -abstract class RoutingNode[T] { - def shardInfo: ShardInfo - def weight: Int - def children: Seq[RoutingNode[T]] - - def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] - def readOperation[A](f: T => A): A - def writeOperation[A](f: T => A): A - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] - - def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { - rebuildRead(Nil) { (shard, toRebuild) => - val result = f(shard) - if (!result.isEmpty) toRebuild.foreach(rebuild(shard, _)) - result - } match { - case Left(s) => None - case Right(rv) => Some(rv) - } - } - - protected def normalizeException(ex: Throwable, shardId: ShardId): Option[Throwable] = ex match { - case e: ExecutionException => normalizeException(e.getCause, shardId) - // fondly known as JavaOutrageException - case e: UndeclaredThrowableException => normalizeException(e.getCause, shardId) - case e: ShardBlackHoleException => None - case e: TimeoutException => Some(new ReplicatingShardTimeoutException(shardId, e)) - case e => Some(e) - } -} - - -class PassThroughNode[T](val shardInfo: ShardInfo, val weight: Int, val children: Seq[RoutingNode[T]]) -extends RoutingNode[T] { - - val inner = children.head - - def readAllOperation[A](f: T => A) = inner.readAllOperation(f) - def readOperation[A](f: T => A) = inner.readOperation(f) - def writeOperation[A](f: T => A) = inner.writeOperation(f) - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { - inner.rebuildRead(toRebuild)(f) - } -} - - -class LeafRoutingNode[T](val shardInfo: ShardInfo, val weight: Int, shard: T) extends RoutingNode[T] { - def this(weight: Int, shard: T) = this(new ShardInfo(new ShardId("", ""), "", "", "", Busy.Normal), weight, shard) - - val children: Seq[RoutingNode[T]] = Seq() - - def readAllOperation[A](f: T => A) = Seq(try { Right(f(shard)) } catch { case e => Left(e) }) - def readOperation[A](f: T => A) = f(shard) - def writeOperation[A](f: T => A) = f(shard) - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { - f(shard, toRebuild) match { - case Some(rv) => Right(rv) - case None => Left(shard :: toRebuild) - } - } -} - -class LeafRoutingNodeFactory[T](shardFactory: ShardFactory[T]) extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new LeafRoutingNode(shardInfo, weight, shardFactory.instantiate(shardInfo)) - } - - override def materialize(shardInfo: ShardInfo) { - shardFactory.materialize(shardInfo) - } -} diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala new file mode 100644 index 00000000..df7980fd --- /dev/null +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -0,0 +1,42 @@ +package com.twitter.gizzard.shards + +import java.lang.reflect.UndeclaredThrowableException +import java.util.concurrent.{ExecutionException, TimeoutException} + + +abstract class RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]): RoutingNode[T] + def materialize(shardInfo: ShardInfo) {} +} + +abstract class RoutingNode[T] { + def shardInfo: ShardInfo + def weight: Int + def children: Seq[RoutingNode[T]] + + def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] + def readOperation[A](f: T => A): A + def writeOperation[A](f: T => A): A + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] + + def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { + rebuildRead(Nil) { (shard, toRebuild) => + val result = f(shard) + if (!result.isEmpty) toRebuild.foreach(rebuild(shard, _)) + result + } match { + case Left(s) => None + case Right(rv) => Some(rv) + } + } + + protected def normalizeException(ex: Throwable, shardId: ShardId): Option[Throwable] = ex match { + case e: ExecutionException => normalizeException(e.getCause, shardId) + // fondly known as JavaOutrageException + case e: UndeclaredThrowableException => normalizeException(e.getCause, shardId) + case e: ShardBlackHoleException => None + case e: TimeoutException => Some(new ReplicatingShardTimeoutException(shardId, e)) + case e => Some(e) + } +} diff --git a/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala b/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala index 6c871a33..62c25469 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ShardFactory.scala @@ -2,6 +2,6 @@ package com.twitter.gizzard.shards trait ShardFactory[T] { - def instantiate(shardInfo: ShardInfo): T + def instantiate(shardInfo: ShardInfo, weight: Int): T def materialize(shardInfo: ShardInfo) } diff --git a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala new file mode 100644 index 00000000..b20ba0a3 --- /dev/null +++ b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala @@ -0,0 +1,97 @@ +package com.twitter.gizzard.shards + + +// Base class for all read/write flow wrapper shards + +class WrapperRoutingNode[T](val shardInfo: ShardInfo, val weight: Int, val children: Seq[RoutingNode[T]]) +extends RoutingNode[T] { + + val inner = children.head + + def readAllOperation[A](f: T => A) = inner.readAllOperation(f) + def readOperation[A](f: T => A) = inner.readOperation(f) + def writeOperation[A](f: T => A) = inner.writeOperation(f) + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + inner.rebuildRead(toRebuild)(f) + } +} + + +// BlockedShard. Refuse and fail all traffic. + +class BlockedShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new BlockedShard(shardInfo, weight, children) + } +} + +class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends WrapperRoutingNode[T](shardInfo, weight, children) { + + protected def exception = new ShardRejectedOperationException("shard is offline", shardInfo.id) + + override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]](Left(exception)) + override def readOperation[A](method: T => A) = throw exception + override def writeOperation[A](method: T => A) = throw exception + + override protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + throw exception + } +} + + + +// BlackHoleShard. Silently refuse all traffic. + +class BlackHoleShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new BlackHoleShard(shardInfo, weight, children) + } +} + +class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends BlockedShard[T](shardInfo, weight, children) { + + override protected def exception = throw new ShardBlackHoleException(shardInfo.id) + + override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]]() +} + + + +// WriteOnlyShard. Fail all read traffic. + +class WriteOnlyShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new WriteOnlyShard(shardInfo, weight, children) + } +} + +class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends WrapperRoutingNode[T](shardInfo, weight, children) { + + private def exception = new ShardRejectedOperationException("shard is write-only", shardInfo.id) + + override def readAllOperation[A](method: T => A) = Seq(Left(exception)) + override def readOperation[A](method: T => A) = throw exception + override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception +} + + + +// ReadOnlyShard. Fail all write traffic. + +class ReadOnlyShardFactory[T] extends RoutingNodeFactory[T] { + def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { + new ReadOnlyShard(shardInfo, weight, children) + } +} + +class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) +extends WrapperRoutingNode[T](shardInfo, weight, children) { + + override def writeOperation[A](method: T => A) = { + throw new ShardRejectedOperationException("shard is read-only", shardInfo.id) + } +} diff --git a/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala b/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala deleted file mode 100644 index 0175c3a9..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/WriteOnlyShard.scala +++ /dev/null @@ -1,19 +0,0 @@ -package com.twitter.gizzard -package shards - - -class WriteOnlyShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: shards.ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new WriteOnlyShard(shardInfo, weight, children) - } -} - -class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends PassThroughNode[T](shardInfo, weight, children) { - - private def exception = new ShardRejectedOperationException("shard is write-only", shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq(Left(exception)) - override def readOperation[A](method: T => A) = throw exception - override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception -} diff --git a/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala b/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala index 517f22f7..ddcdc608 100644 --- a/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala +++ b/src/main/scala/com/twitter/gizzard/test/FakeReadWriteShard.scala @@ -7,5 +7,5 @@ class FakeReadWriteShard[T]( shard: T, shardInfo: ShardInfo, weight: Int, - override val children: Seq[RoutingNode[T]]) -extends LeafRoutingNode[T](shardInfo, weight, shard) + children: Seq[RoutingNode[T]]) +extends LeafRoutingNode[T](shard, shardInfo, weight) diff --git a/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala b/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala index 07884f79..6175b24e 100644 --- a/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala +++ b/src/test/scala/com/twitter/gizzard/fake/NestableShard.scala @@ -1,16 +1,19 @@ -package com.twitter.gizzard -package fake +package com.twitter.gizzard.fake import scala.collection.mutable import org.specs.mock.{ClassMocker, JMocker} -import shards.{ShardException,ShardInfo} -class NestableShardFactory extends shards.ShardFactory[Shard] { - def instantiate(shardInfo: ShardInfo) = new NestableShard(shardInfo) +import com.twitter.gizzard.shards.{ShardFactory,ShardException,ShardInfo} + +class NestableShardFactory extends ShardFactory[Shard] { + def instantiate(shardInfo: ShardInfo, weight: Int) = { + new NestableShard(shardInfo) + } + def materialize(shardInfo: ShardInfo) = () } -class NestableShard(val shardInfo: shards.ShardInfo) extends Shard { +class NestableShard(val shardInfo: ShardInfo) extends Shard { val map = new mutable.HashMap[String, String] def get(key: String) = { @@ -21,5 +24,4 @@ class NestableShard(val shardInfo: shards.ShardInfo) extends Shard { map.put(key, value) value } - } diff --git a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala index 63f7c434..3a9cc53b 100644 --- a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala +++ b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala @@ -158,7 +158,9 @@ class TestShardAdapter(s: shards.RoutingNode[TestShard]) extends TestShard { class SqlShardFactory(qeFactory: QueryEvaluatorFactory, conn: Connection) extends shards.ShardFactory[TestShard] { - def instantiate(info: ShardInfo) = new SqlShard(qeFactory(conn.withHost(info.hostname)), info) + def instantiate(info: ShardInfo, weight: Int) = { + new SqlShard(qeFactory(conn.withHost(info.hostname)), info) + } def materialize(info: ShardInfo) { val ddl = diff --git a/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala index 749eef75..b96277e7 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/NameServerSpec.scala @@ -13,7 +13,7 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc val SQL_SHARD = "com.example.SqlShard" val nameServerShard = mock[nameserver.Shard] - val routingNode = new LeafRoutingNode(1, nameServerShard) + val routingNode = new LeafRoutingNode(nameServerShard, 1) var shardRepository = mock[ShardRepository[AnyRef]] val mappingFunction = (n: Long) => n @@ -33,7 +33,7 @@ object NameServerSpec extends ConfiguredSpecification with JMocker with ClassMoc new Host("host3", 7777, "c2", HostStatus.Normal)) val shard = mock[AnyRef] - val node = new LeafRoutingNode(1, shard) + val node = new LeafRoutingNode(shard, 1) doBefore { expect { diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 3ccd8953..491876c6 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -15,7 +15,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val shard1 = mock[fake.Shard] val shard2 = mock[fake.Shard] val shard3 = mock[fake.Shard] - val List(node1, node2, node3) = List(shard1, shard2, shard3).map(new LeafRoutingNode(1, _)) + val List(node1, node2, node3) = List(shard1, shard2, shard3).map(new LeafRoutingNode(_, 1)) val future = new Future("Future!", 1, 1, 1.second, 1.second) val shards = List(node1, node2) @@ -168,7 +168,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val shardInfo = new ShardInfo("fake", "fake", "localhost") val mock1 = mock[EnufShard] val mock2 = mock[EnufShard] - val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(1, _)) + val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(_, 1)) val shards = List(node1, node2) val loadBalancer = () => shards val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards, loadBalancer, Some(future)) diff --git a/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala index 89ff7346..0e779052 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ShardsIntegrationSpec.scala @@ -28,7 +28,7 @@ object ShardsIntegrationSpec extends ConfiguredSpecification with JMocker with C } val factory = new ShardFactory[UserShard] { - def instantiate(shardInfo: ShardInfo) = { + def instantiate(shardInfo: ShardInfo, weight: Int) = { new UserShard(shardInfo) } @@ -47,7 +47,7 @@ object ShardsIntegrationSpec extends ConfiguredSpecification with JMocker with C shardRepository += (("com.example.UserShard", factory)) shardRepository += (("com.example.SqlShard", factory)) reset(queryEvaluator) - nameServerShard = new LeafRoutingNode(1, new SqlShard(queryEvaluator)) + nameServerShard = new LeafRoutingNode(new SqlShard(queryEvaluator), 1) nameServer = new NameServer(nameServerShard, shardRepository, NullJobRelayFactory, mapping) nameServer.reload() From d9eb4e1d3ceabeeb3941bf70182d5e3cc3745b3e Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 28 Apr 2011 11:03:05 -0700 Subject: [PATCH 04/16] golf --- .../gizzard/nameserver/ShardRepository.scala | 12 +++-- .../twitter/gizzard/shards/RoutingNode.scala | 6 +++ .../twitter/gizzard/shards/ShardInfo.scala | 30 ++++++++---- .../gizzard/shards/WrapperRoutingNodes.scala | 47 +++++-------------- .../nameserver/ShardRepositorySpec.scala | 19 ++++---- 5 files changed, 58 insertions(+), 56 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala index c3d38d76..dfb415fa 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala @@ -17,6 +17,10 @@ class ShardRepository[T] { nodeFactories += (className -> factory) } + def addRoutingNode(className: String, cons: (ShardInfo, Int, Seq[RoutingNode[T]]) => RoutingNode[T]) { + addRoutingNode(className, new ConstructorRoutingNodeFactory(cons)) + } + def find(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { factory(shardInfo.className).instantiate(shardInfo, weight, children) } @@ -48,10 +52,10 @@ extends ShardRepository[T] { def setupPackage(packageName: String) { val prefix = if (packageName == "") packageName else packageName + "." - addRoutingNode(prefix + "ReadOnlyShard", new shards.ReadOnlyShardFactory[T]) - addRoutingNode(prefix + "BlockedShard", new shards.BlockedShardFactory[T]) - addRoutingNode(prefix + "WriteOnlyShard", new shards.WriteOnlyShardFactory[T]) - addRoutingNode(prefix + "BlackHoleShard", new shards.BlackHoleShardFactory[T]) + addRoutingNode(prefix + "ReadOnlyShard", ReadOnlyShard[T] _) + addRoutingNode(prefix + "BlockedShard", BlockedShard[T] _) + addRoutingNode(prefix + "WriteOnlyShard", WriteOnlyShard[T] _) + addRoutingNode(prefix + "BlackHoleShard", BlackHoleShard[T] _) addRoutingNode(prefix + "ReplicatingShard", new shards.ReplicatingShardFactory[T](replicationFuture)) addRoutingNode(prefix + "FailingOverShard", new shards.FailingOverShardFactory[T](replicationFuture)) } diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index df7980fd..720ffda2 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -9,6 +9,12 @@ abstract class RoutingNodeFactory[T] { def materialize(shardInfo: ShardInfo) {} } +// Turn case class or other generic constructors into node factories. +class ConstructorRoutingNodeFactory[T](constructor: (ShardInfo, Int, Seq[RoutingNode[T]]) => RoutingNode[T]) +extends RoutingNodeFactory[T] { + def instantiate(info: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = constructor(info, weight, children) +} + abstract class RoutingNode[T] { def shardInfo: ShardInfo def weight: Int diff --git a/src/main/scala/com/twitter/gizzard/shards/ShardInfo.scala b/src/main/scala/com/twitter/gizzard/shards/ShardInfo.scala index 2a423ecb..d8d0126b 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ShardInfo.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ShardInfo.scala @@ -4,18 +4,32 @@ package shards import thrift.conversions.ShardId._ -case class ShardInfo(var id: ShardId, var className: String, - var sourceType: String, var destinationType: String, var busy: Busy.Value) { - def hostname = id.hostname - def tablePrefix = id.tablePrefix +case class ShardInfo( + var id: ShardId, + var className: String, + var sourceType: String, + var destinationType: String, + var busy: Busy.Value +) { - def this(className: String, tablePrefix: String, hostname: String) = + def this(className: String, tablePrefix: String, hostname: String) = { this(ShardId(hostname, tablePrefix), className, "", "", Busy.Normal) + } - def this(className: String, tablePrefix: String, hostname: String, sourceType: String, - destinationType: String) = + def this( + className: String, + tablePrefix: String, + hostname: String, + sourceType: String, + destinationType: String + ) = { this(ShardId(hostname, tablePrefix), className, sourceType, destinationType, Busy.Normal) + } + + def hostname = id.hostname + def tablePrefix = id.tablePrefix - override def clone(): ShardInfo = + override def clone(): ShardInfo = { new ShardInfo(id, className, sourceType, destinationType, busy) + } } diff --git a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala index b20ba0a3..3ec5c134 100644 --- a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala +++ b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala @@ -3,8 +3,7 @@ package com.twitter.gizzard.shards // Base class for all read/write flow wrapper shards -class WrapperRoutingNode[T](val shardInfo: ShardInfo, val weight: Int, val children: Seq[RoutingNode[T]]) -extends RoutingNode[T] { +abstract class WrapperRoutingNode[T] extends RoutingNode[T] { val inner = children.head @@ -20,14 +19,7 @@ extends RoutingNode[T] { // BlockedShard. Refuse and fail all traffic. -class BlockedShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new BlockedShard(shardInfo, weight, children) - } -} - -class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends WrapperRoutingNode[T](shardInfo, weight, children) { +case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { protected def exception = new ShardRejectedOperationException("shard is offline", shardInfo.id) @@ -44,32 +36,24 @@ extends WrapperRoutingNode[T](shardInfo, weight, children) { // BlackHoleShard. Silently refuse all traffic. -class BlackHoleShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new BlackHoleShard(shardInfo, weight, children) - } -} +case class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { -class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends BlockedShard[T](shardInfo, weight, children) { - - override protected def exception = throw new ShardBlackHoleException(shardInfo.id) + protected def exception = throw new ShardBlackHoleException(shardInfo.id) override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]]() + override def readOperation[A](method: T => A) = throw exception + override def writeOperation[A](method: T => A) = throw exception + + override protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { + throw exception + } } // WriteOnlyShard. Fail all read traffic. -class WriteOnlyShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new WriteOnlyShard(shardInfo, weight, children) - } -} - -class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends WrapperRoutingNode[T](shardInfo, weight, children) { +case class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { private def exception = new ShardRejectedOperationException("shard is write-only", shardInfo.id) @@ -82,14 +66,7 @@ extends WrapperRoutingNode[T](shardInfo, weight, children) { // ReadOnlyShard. Fail all write traffic. -class ReadOnlyShardFactory[T] extends RoutingNodeFactory[T] { - def instantiate(shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = { - new ReadOnlyShard(shardInfo, weight, children) - } -} - -class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) -extends WrapperRoutingNode[T](shardInfo, weight, children) { +case class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { override def writeOperation[A](method: T => A) = { throw new ShardRejectedOperationException("shard is read-only", shardInfo.id) diff --git a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala index 772ea924..c7e0a330 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala @@ -5,28 +5,29 @@ import com.twitter.conversions.time._ import org.specs.Specification import org.specs.mock.{ClassMocker, JMocker} -import com.twitter.gizzard.shards.RoutingNode +import com.twitter.gizzard.shards.{RoutingNode, ShardInfo} object ShardRepositorySpec extends ConfiguredSpecification with JMocker with ClassMocker { "BasicShardRepository" should { + val info = new ShardInfo("", "", "") val future = mock[Future] - val shard = mock[RoutingNode[AnyRef]] - val repository = new BasicShardRepository(Some(future)) + val shard = Seq(mock[RoutingNode[AnyRef]]) + val repository = new BasicShardRepository[AnyRef](Some(future)) "find a read-only shard" in { - repository.factory("ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[AnyRef]] - repository.factory("com.twitter.gizzard.shards.ReadOnlyShard") must haveClass[shards.ReadOnlyShardFactory[AnyRef]] + repository.factory("ReadOnlyShard").instantiate(info, 1, shard) must haveClass[shards.ReadOnlyShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.ReadOnlyShard").instantiate(info, 1, shard) must haveClass[shards.ReadOnlyShard[AnyRef]] repository.factory("com.example.bogis.ReadOnlyShard") must throwA[NoSuchElementException] } "find a write-only shard" in { - repository.factory("WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[AnyRef]] - repository.factory("com.twitter.gizzard.shards.WriteOnlyShard") must haveClass[shards.WriteOnlyShardFactory[AnyRef]] + repository.factory("WriteOnlyShard").instantiate(info, 1, shard) must haveClass[shards.WriteOnlyShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.WriteOnlyShard").instantiate(info, 1, shard) must haveClass[shards.WriteOnlyShard[AnyRef]] repository.factory("com.example.bogis.WriteOnlyShard") must throwA[NoSuchElementException] } "find a blocked shard" in { - repository.factory("BlockedShard") must haveClass[shards.BlockedShardFactory[AnyRef]] - repository.factory("com.twitter.gizzard.shards.BlockedShard") must haveClass[shards.BlockedShardFactory[AnyRef]] + repository.factory("BlockedShard").instantiate(info, 1, shard) must haveClass[shards.BlockedShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.BlockedShard").instantiate(info, 1, shard) must haveClass[shards.BlockedShard[AnyRef]] repository.factory("com.example.bogis.BlockedShard") must throwA[NoSuchElementException] } "find a replicating shard" in { From 0468943e7c818e5cc8e36034082df5a1cd02d693 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Tue, 3 May 2011 15:10:54 -0700 Subject: [PATCH 05/16] add ability to skip some shards. --- .../twitter/gizzard/shards/ReplicatingShard.scala | 12 ++++++++++++ .../com/twitter/gizzard/shards/RoutingNode.scala | 8 ++++++++ .../twitter/gizzard/shards/WrapperRoutingNodes.scala | 3 +-- 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index 86d93612..615f4de4 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -35,6 +35,18 @@ extends RoutingNode[T] { lazy val log = Logger.get + override def skipShard(ss: ShardId*) = { + val filtered = children.filterNot(ss.toSet.contains) + + if (filtered.isEmpty) { + BlackHoleShard(shardInfo, weight, children) + } else if (filtered.size == children.size) { + this + } else { + new ReplicatingShard[T](shardInfo, weight, filtered, new LoadBalancer(filtered), future) + } + } + def readAllOperation[A](f: T => A) = fanout(children)(_.readAllOperation(f)) def readOperation[A](f: T => A) = failover(loadBalancer())(_.readOperation(f)) diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index 720ffda2..150f013a 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -24,6 +24,14 @@ abstract class RoutingNode[T] { def readOperation[A](f: T => A): A def writeOperation[A](f: T => A): A + + def skipShard(ss: ShardId*): RoutingNode[T] = if (ss.toSet.contains(shardInfo.id)) { + BlackHoleShard(shardInfo, weight, children) + } else { + this + } + + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { diff --git a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala index 3ec5c134..567ffb97 100644 --- a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala +++ b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala @@ -5,7 +5,7 @@ package com.twitter.gizzard.shards abstract class WrapperRoutingNode[T] extends RoutingNode[T] { - val inner = children.head + lazy val inner = children.head def readAllOperation[A](f: T => A) = inner.readAllOperation(f) def readOperation[A](f: T => A) = inner.readOperation(f) @@ -16,7 +16,6 @@ abstract class WrapperRoutingNode[T] extends RoutingNode[T] { } } - // BlockedShard. Refuse and fail all traffic. case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { From 5ff57b7d47af2216929c0217d928f3678a955b30 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Wed, 11 May 2011 11:41:13 -0700 Subject: [PATCH 06/16] add ability to skip shards by shardId --- .../twitter/gizzard/shards/ReplicatingShard.scala | 5 +++-- .../com/twitter/gizzard/shards/RoutingNode.scala | 4 +--- .../gizzard/shards/ReplicatingShardSpec.scala | 12 +++++++++++- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index 615f4de4..fb2ddee8 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -36,10 +36,11 @@ extends RoutingNode[T] { lazy val log = Logger.get override def skipShard(ss: ShardId*) = { - val filtered = children.filterNot(ss.toSet.contains) + val toSkip = ss.toSet + val filtered = children.filterNot { c => toSkip contains c.shardInfo.id } if (filtered.isEmpty) { - BlackHoleShard(shardInfo, weight, children) + BlackHoleShard(shardInfo, weight, Seq(this)) } else if (filtered.size == children.size) { this } else { diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index 150f013a..13ea102d 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -24,14 +24,12 @@ abstract class RoutingNode[T] { def readOperation[A](f: T => A): A def writeOperation[A](f: T => A): A - def skipShard(ss: ShardId*): RoutingNode[T] = if (ss.toSet.contains(shardInfo.id)) { - BlackHoleShard(shardInfo, weight, children) + BlackHoleShard(shardInfo, weight, Seq(this)) } else { this } - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 491876c6..e8912fe1 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -15,7 +15,9 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val shard1 = mock[fake.Shard] val shard2 = mock[fake.Shard] val shard3 = mock[fake.Shard] - val List(node1, node2, node3) = List(shard1, shard2, shard3).map(new LeafRoutingNode(_, 1)) + val List(node1, node2, node3) = List(shard1, shard2, shard3).zipWithIndex map { case (s, i) => + new LeafRoutingNode(s, new ShardInfo("", "shard"+ (i + 1), "fake"), 1) + } val future = new Future("Future!", 1, 1, 1.second, 1.second) val shards = List(node1, node2) @@ -24,6 +26,14 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val replicatingShardInfo = new ShardInfo("", "replicating_shard", "hostname") var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, Some(future)) + "filters shards" in { + expect { + one(shard2).get("name").willReturn(Some("bob")) + } + + replicatingShard.skipShard(ShardId("fake", "shard1")).readOperation(_.get("name")) mustEqual Some("bob") + } + "read failover" in { "when shard1 throws an exception" in { val shard1Info = new ShardInfo("", "table_prefix", "hostname") From bf9316f247720d8feb8370fe0ef1e4377593316b Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Wed, 1 Jun 2011 22:44:50 -0700 Subject: [PATCH 07/16] add NodeSet --- .../com/twitter/gizzard/shards/NodeSet.scala | 103 ++++++++++++++++++ .../twitter/gizzard/shards/RoutingNode.scala | 17 ++- 2 files changed, 115 insertions(+), 5 deletions(-) create mode 100644 src/main/scala/com/twitter/gizzard/shards/NodeSet.scala diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala new file mode 100644 index 00000000..da02e387 --- /dev/null +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -0,0 +1,103 @@ +package com.twitter.gizzard.shards + +import scala.collection.generic.CanBuild +import com.twitter.util.{Try, Throw} + +// For read or write, three node states: +// - normal: should apply normally +// - block: should indicate error +// - skip: should ignore the shard. + +// skip(ShardId).read.any(T => R) => R + +// read.blockedShards => Seq[ShardInfo] +// read.iterator => Iterator[T] +// read.map(T => R) => Seq[R] // throws error if block exists. +// read.foreach(T => R) => Seq[R] // throws error if block exists. +// read.all(T => R) => Seq[Try[R]] +// read.any(T => R) => R +// write.all(T => R) => Seq[Try[R]] + +// iterator => Iterator[(T, ShardInfo)] +// map((T, ShardInfo) => R) => Seq[R] +// all((T, ShardInfo) => R) => Seq[Try[R]] + +// withInfo -> RoutingNode[(T, ShardInfo)] + + +trait NodeIterable[T] { + def rootInfo: ShardInfo + def activeShards: Seq[(ShardInfo, T)] + def blockedShards: Seq[ShardInfo] + + def containsBlocked = !blockedShards.isEmpty + + def any[R](f: T => R): R = { + if (activeShards.isEmpty && blockedShards.isEmpty) { + throw new ShardBlackHoleException(rootInfo.id) + } + + _any(f, iterator) + } + + protected final def _any[R](f: T => R, iter: Iterator[T]): R = { + while (iter.hasNext) { + try { + return f(iter.next) + } catch { + case e: ShardException => () + } + } + + throw new ShardOfflineException(rootInfo.id) + } + + def all[R, That](f: T => R)(implicit bf: CanBuild[Try[R], That] = Seq.canBuildFrom[Try[R]]): That = { + val b = bf() + for ((i, s) <- activeShards) b += Try(f(s)) + for (s <- blockedShards) b += Throw(new ShardOfflineException(s.id)) + b.result + } + + // iterators are lazy, so map works here. + def iterator: Iterator[T] = activeShards.iterator map { case (i, s) => s } + + // throws error if block exists + def map[R, That](f: T => R)(implicit bf: CanBuild[R, That]): That = { + val b = bf() + for (s <- this) b += f(s) + b.result + } + + // throws error if block exists + def flatMap[R, That](f: T => Traversable[R])(implicit bf: CanBuild[R, That]): That = { + val b = bf() + for (s <- this) b ++= f(s) + b.result + } + + def foreach[U](f: T => U) { + all(f) foreach { + case Throw(e) => throw e + case _ => () + } + } +} + +class NodeSet[T]( + val rootInfo: ShardInfo, // XXX: replace with forwarding id. + val activeShards: Seq[(ShardInfo, T)], + val blockedShards: Seq[ShardInfo]) +extends NodeIterable[T] { + + def filter(f: (ShardInfo, Option[T]) => Boolean) = { + val activeFiltered = activeShards filter { case (i, s) => f(i, Some(s)) } + val blockedFiltered = blockedShards filter { i => f(i, None) } + new NodeSet(rootInfo, activeFiltered, blockedFiltered) + } + + def skip(ss: ShardId*) = { + val set = ss.toSet + filter { (info, _) => set contains info.id } + } +} diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index 13ea102d..77d93045 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -15,23 +15,30 @@ extends RoutingNodeFactory[T] { def instantiate(info: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = constructor(info, weight, children) } + abstract class RoutingNode[T] { def shardInfo: ShardInfo def weight: Int def children: Seq[RoutingNode[T]] - def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] - def readOperation[A](f: T => A): A - def writeOperation[A](f: T => A): A - - def skipShard(ss: ShardId*): RoutingNode[T] = if (ss.toSet.contains(shardInfo.id)) { + def skip(ss: ShardId*): RoutingNode[T] = if (ss.toSet.contains(shardInfo.id)) { BlackHoleShard(shardInfo, weight, Seq(this)) } else { this } + @deprecated("use read.all instead") + def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] + + @deprecated("use read.any instead") + def readOperation[A](f: T => A): A + + @deprecated("use write.all instead") + def writeOperation[A](f: T => A): A + protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] + @deprecated("reimplement using read.iterator instead") def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { rebuildRead(Nil) { (shard, toRebuild) => val result = f(shard) From dfcd66ac4fe515e219cc9dac2d3131cd26bca08b Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 14:59:19 -0700 Subject: [PATCH 08/16] refactor RoutingNode and subclasses to use new NodeSet api --- .../gizzard/shards/LeafRoutingNode.scala | 6 +- .../gizzard/shards/ReplicatingShard.scala | 83 +------------------ .../twitter/gizzard/shards/RoutingNode.scala | 73 +++++++++++++--- .../gizzard/shards/WrapperRoutingNodes.scala | 44 ++-------- .../gizzard/shards/ReplicatingShardSpec.scala | 44 +++++----- 5 files changed, 96 insertions(+), 154 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala index a82a807b..8fe800ad 100644 --- a/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/LeafRoutingNode.scala @@ -3,14 +3,14 @@ package com.twitter.gizzard.shards class LeafRoutingNode[T](shard: T, val shardInfo: ShardInfo, val weight: Int) extends RoutingNode[T] { + import RoutingNode._ + val children = Nil // convenience constructor for manual creation. def this(shard: T, weight: Int) = this(shard, new ShardInfo("", "", ""), weight) - def readAllOperation[A](f: T => A) = Seq(try { Right(f(shard)) } catch { case e => Left(e) }) - def readOperation[A](f: T => A) = f(shard) - def writeOperation[A](f: T => A) = f(shard) + protected[shards] def collectedShards = Seq(Leaf(shardInfo, Allow, Allow, shard)) protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { f(shard, toRebuild) match { diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index 77991ec3..c15cd565 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -33,9 +33,11 @@ class ReplicatingShard[T]( val future: Option[Future]) extends RoutingNode[T] { - lazy val log = Logger.get + import RoutingNode._ - override def skipShard(ss: ShardId*) = { + protected[shards] def collectedShards = loadBalancer() flatMap { _.collectedShards } + + override def skip(ss: ShardId*) = { val toSkip = ss.toSet val filtered = children.filterNot { c => toSkip contains c.shardInfo.id } @@ -47,81 +49,4 @@ extends RoutingNode[T] { new ReplicatingShard[T](shardInfo, weight, filtered, new LoadBalancer(filtered), future) } } - - def readAllOperation[A](f: T => A) = fanout(children)(_.readAllOperation(f)) - - def readOperation[A](f: T => A) = failover(loadBalancer())(_.readOperation(f)) - - def writeOperation[A](f: T => A) = { - val allResults = fanout(children) { c => - try { - Seq(Right(c.writeOperation(f))) - } catch { - case e => normalizeException(e, shardInfo.id).map(Left(_)).toSeq - } - } map { - case Left(e) => throw e - case Right(result) => result - } - - allResults.headOption getOrElse { - throw new ShardBlackHoleException(shardInfo.id) - } - } - - protected def fanout[A](replicas: Seq[RoutingNode[T]])(f: RoutingNode[T] => Seq[Either[Throwable,A]]) = { - future match { - case None => replicas flatMap f - case Some(future) => { - replicas.map { r => Pair(r, future(f(r))) } flatMap { case (replica, task) => - try { - task.get(future.timeout.inMillis, TimeUnit.MILLISECONDS) - } catch { - case e => normalizeException(e, replica.shardInfo.id).map(Left(_)).toSeq - } - } - } - } - } - - protected def failover[A](replicas: Seq[RoutingNode[T]])(f: RoutingNode[T] => A): A = { - replicas foreach { replica => - try { - return f(replica) - } catch { - case e: ShardRejectedOperationException => () - case e: ShardException => log.warning(e, "Error on %s: %s", replica.shardInfo.id, e) - } - } - - throw new ShardOfflineException(shardInfo.id) - } - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] = { - val start: Either[List[T],A] = Left(toRebuild) - var everSuccessful = false - - val rv = (children foldLeft start) { (result, replica) => - result match { - case Right(rv) => return Right(rv) - case Left(toRebuild) => try { - val next = replica.rebuildRead(toRebuild)(f) - everSuccessful = true - next - } catch { - case e: ShardRejectedOperationException => Left(toRebuild) - case e: ShardException => { - log.warning(e, "Error on %s: %s", replica.shardInfo.id, e) - Left(toRebuild) - } - } - } - } - - if (!everSuccessful) { - throw new ShardOfflineException(shardInfo.id) - } else { - rv - } - } } diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index 77d93045..9ee04567 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -2,6 +2,7 @@ package com.twitter.gizzard.shards import java.lang.reflect.UndeclaredThrowableException import java.util.concurrent.{ExecutionException, TimeoutException} +import com.twitter.util.{Try, Return, Throw} abstract class RoutingNodeFactory[T] { @@ -15,12 +16,35 @@ extends RoutingNodeFactory[T] { def instantiate(info: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) = constructor(info, weight, children) } +protected[shards] object RoutingNode { + // XXX: use real behavior once ShardStatus lands + sealed trait Behavior + case object Allow extends Behavior + case object Deny extends Behavior + case object Ignore extends Behavior + case class Leaf[T](info: ShardInfo, readBehavior: Behavior, writeBehavior: Behavior, shard: T) +} abstract class RoutingNode[T] { def shardInfo: ShardInfo def weight: Int def children: Seq[RoutingNode[T]] + import RoutingNode._ + + protected[shards] def collectedShards: Seq[Leaf[T]] + + protected def nodeSetFromCollected(filter: Leaf[T] => Behavior) = { + val m = collectedShards groupBy filter + val active = m.getOrElse(Allow, Nil) map { l => (l.info, l.shard) } + val blocked = m.getOrElse(Deny, Nil) map { _.info } + new NodeSet(shardInfo, active, blocked) + } + + def read = nodeSetFromCollected { _.readBehavior } + + def write = nodeSetFromCollected { _.writeBehavior } + def skip(ss: ShardId*): RoutingNode[T] = if (ss.toSet.contains(shardInfo.id)) { BlackHoleShard(shardInfo, weight, Seq(this)) } else { @@ -28,25 +52,50 @@ abstract class RoutingNode[T] { } @deprecated("use read.all instead") - def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] + def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] = read.all(f) map { + case Return(r) => Right(r) + case Throw(e) => Left(e) + } @deprecated("use read.any instead") - def readOperation[A](f: T => A): A + def readOperation[A](f: T => A) = read.any(f) @deprecated("use write.all instead") - def writeOperation[A](f: T => A): A - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]): Either[List[T],A] + def writeOperation[A](f: T => A) = { + var rv: Option[A] = None + write foreach { s => rv = Some(f(s)) } + rv.getOrElse(throw new ShardBlackHoleException(shardInfo.id)) + } @deprecated("reimplement using read.iterator instead") def rebuildableReadOperation[A](f: T => Option[A])(rebuild: (T, T) => Unit): Option[A] = { - rebuildRead(Nil) { (shard, toRebuild) => - val result = f(shard) - if (!result.isEmpty) toRebuild.foreach(rebuild(shard, _)) - result - } match { - case Left(s) => None - case Right(rv) => Some(rv) + val iter = read.iterator + + var everSuccessful = false + var toRebuild: List[T] = Nil + + while (iter.hasNext) { + val shard = iter.next + + try { + val result = f(shard) + everSuccessful = true + + if (result.isEmpty) { + toRebuild = shard :: toRebuild + } else { + toRebuild.foreach(rebuild(shard, _)) + return result + } + } catch { + case e: ShardException => () // XXX: log error + } + } + + if (everSuccessful) { + None + } else { + throw new ShardOfflineException(shardInfo.id) } } diff --git a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala index 567ffb97..82334867 100644 --- a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala +++ b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala @@ -5,30 +5,15 @@ package com.twitter.gizzard.shards abstract class WrapperRoutingNode[T] extends RoutingNode[T] { - lazy val inner = children.head + protected def leafTransform(l: RoutingNode.Leaf[T]): RoutingNode.Leaf[T] - def readAllOperation[A](f: T => A) = inner.readAllOperation(f) - def readOperation[A](f: T => A) = inner.readOperation(f) - def writeOperation[A](f: T => A) = inner.writeOperation(f) - - protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { - inner.rebuildRead(toRebuild)(f) - } + protected[shards] def collectedShards = children flatMap { _.collectedShards.map(leafTransform) } } // BlockedShard. Refuse and fail all traffic. case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { - - protected def exception = new ShardRejectedOperationException("shard is offline", shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]](Left(exception)) - override def readOperation[A](method: T => A) = throw exception - override def writeOperation[A](method: T => A) = throw exception - - override protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { - throw exception - } + protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(readBehavior = RoutingNode.Deny, writeBehavior = RoutingNode.Deny) } @@ -36,16 +21,7 @@ case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Rout // BlackHoleShard. Silently refuse all traffic. case class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { - - protected def exception = throw new ShardBlackHoleException(shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq[Either[Throwable,A]]() - override def readOperation[A](method: T => A) = throw exception - override def writeOperation[A](method: T => A) = throw exception - - override protected[shards] def rebuildRead[A](toRebuild: List[T])(f: (T, Seq[T]) => Option[A]) = { - throw exception - } + protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(readBehavior = RoutingNode.Ignore, writeBehavior = RoutingNode.Ignore) } @@ -53,12 +29,7 @@ case class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Ro // WriteOnlyShard. Fail all read traffic. case class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { - - private def exception = new ShardRejectedOperationException("shard is write-only", shardInfo.id) - - override def readAllOperation[A](method: T => A) = Seq(Left(exception)) - override def readOperation[A](method: T => A) = throw exception - override def rebuildableReadOperation[A](method: T => Option[A])(rebuild: (T, T) => Unit) = throw exception + protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(readBehavior = RoutingNode.Deny) } @@ -66,8 +37,5 @@ case class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Ro // ReadOnlyShard. Fail all write traffic. case class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { - - override def writeOperation[A](method: T => A) = { - throw new ShardRejectedOperationException("shard is read-only", shardInfo.id) - } + protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(writeBehavior = RoutingNode.Deny) } diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 322e289e..f63c1ae7 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -8,6 +8,8 @@ import com.twitter.gizzard.nameserver.LoadBalancer object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { + def blackhole[T](n: RoutingNode[T]) = new BlackHoleShard(new ShardInfo("", "", ""), 1, Seq(n)) + "ReplicatingShard" should { val shardId = ShardId("fake", "shard") val shard1 = mock[fake.Shard] @@ -20,16 +22,15 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val future = new Future("Future!", 1, 1, 1.second, 1.second) val shards = List(node1, node2) - def loadBalancer() = shards val replicatingShardInfo = new ShardInfo("", "replicating_shard", "hostname") - var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, Some(future)) + var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, () => shards, Some(future)) "filters shards" in { expect { one(shard2).get("name").willReturn(Some("bob")) } - replicatingShard.skipShard(ShardId("fake", "shard1")).readOperation(_.get("name")) mustEqual Some("bob") + replicatingShard.skip(ShardId("fake", "shard1")).readOperation(_.get("name")) mustEqual Some("bob") } "read failover" in { @@ -115,23 +116,23 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { "when one replica is black holed" in { expect { - one(shard1).put("name", "alice") willThrow new ShardBlackHoleException(shardId) one(shard2).put("name", "alice") } - replicatingShard.writeOperation(_.put("name", "alice")) + + val ss = List(blackhole(node1), node2) + val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) + holed.writeOperation(_.put("name", "alice")) } "when all replicas are black holed" in { - expect { - one(shard1).put("name", "alice") willThrow new ShardBlackHoleException(shardId) - one(shard2).put("name", "alice") willThrow new ShardBlackHoleException(shardId) - } - replicatingShard.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] + val ss = shards.map(blackhole) + val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) + holed.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] } } "in series" in { - var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, loadBalancer, None) + var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, () => shards, None) "normal" in { expect { @@ -149,20 +150,20 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { replicatingShard.writeOperation(_.put("name", "carol")) must throwA[ShardException] } - "with a black hole" in { + "when one replica is black holed" in { expect { - one(shard1).put("name", "carol") willThrow new ShardBlackHoleException(shardId) - one(shard2).put("name", "carol") + one(shard2).put("name", "alice") } - replicatingShard.writeOperation(_.put("name", "carol")) + + val ss = List(blackhole(node1), node2) + val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) + holed.writeOperation(_.put("name", "alice")) } "with all black holes" in { - expect { - one(shard1).put("name", "carol") willThrow new ShardBlackHoleException(shardId) - one(shard2).put("name", "carol") willThrow new ShardBlackHoleException(shardId) - } - replicatingShard.writeOperation(_.put("name", "carol")) must throwA[ShardBlackHoleException] + val ss = shards.map(blackhole) + val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) + holed.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] } } } @@ -178,8 +179,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val mock2 = mock[EnufShard] val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(_, 1)) val shards = List(node1, node2) - val loadBalancer = () => shards - val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards, loadBalancer, Some(future)) + val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards, () => shards, Some(future)) "first shard has data" in { expect { From 8a93a21576a06045793c62302119d108b4e3afe0 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 16:09:14 -0700 Subject: [PATCH 09/16] refactor use of readOperation and writeOperation --- .../gizzard/nameserver/NameServer.scala | 90 +++++++++---------- .../gizzard/fake/ReadWriteShardAdapter.scala | 4 +- .../com/twitter/gizzard/fake/Shard.scala | 2 +- .../gizzard/integration/TestServer.scala | 8 +- .../gizzard/shards/ReplicatingShardSpec.scala | 43 +++++---- 5 files changed, 80 insertions(+), 67 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala index 91d07f08..cdd62603 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/NameServer.scala @@ -69,9 +69,9 @@ class NameServer[T]( familyTree.getOrElse(id, new mutable.ArrayBuffer[LinkInfo]) } - def dumpStructure(tableIds: Seq[Int]) = nameServerShard.readOperation(_.dumpStructure(tableIds)) + def dumpStructure(tableIds: Seq[Int]) = nameServerShard.read.any(_.dumpStructure(tableIds)) - private def currentState() = nameServerShard.readOperation(_.currentState()) + private def currentState() = nameServerShard.read.any(_.currentState()) private def recreateInternalShardState() { val newShardInfos = mutable.Map[ShardId, ShardInfo]() @@ -111,7 +111,7 @@ class NameServer[T]( def reload() { log.info("Loading name server configuration...") - nameServerShard.writeOperation(_.reload()) + nameServerShard.write.foreach(_.reload()) val newRemoteClusters = mutable.Map[String, List[Host]]() @@ -190,151 +190,151 @@ class NameServer[T]( } @throws(classOf[shards.ShardException]) - def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) = { - nameServerShard.writeOperation(_.createShard(shardInfo, repository)) + def createShard[T](shardInfo: ShardInfo, repository: ShardRepository[T]) { + nameServerShard.write.foreach(_.createShard(shardInfo, repository)) } @throws(classOf[shards.ShardException]) def getShard(id: ShardId) = { - nameServerShard.readOperation(_.getShard(id)) + nameServerShard.read.any(_.getShard(id)) } @throws(classOf[shards.ShardException]) - def deleteShard(id: ShardId) = { - nameServerShard.writeOperation(_.deleteShard(id)) + def deleteShard(id: ShardId) { + nameServerShard.write.foreach(_.deleteShard(id)) } @throws(classOf[shards.ShardException]) - def addLink(upId: ShardId, downId: ShardId, weight: Int) = { - nameServerShard.writeOperation(_.addLink(upId, downId, weight)) + def addLink(upId: ShardId, downId: ShardId, weight: Int) { + nameServerShard.write.foreach(_.addLink(upId, downId, weight)) } @throws(classOf[shards.ShardException]) - def removeLink(upId: ShardId, downId: ShardId) = { - nameServerShard.writeOperation(_.removeLink(upId, downId)) + def removeLink(upId: ShardId, downId: ShardId) { + nameServerShard.write.foreach(_.removeLink(upId, downId)) } @throws(classOf[shards.ShardException]) def listUpwardLinks(id: ShardId) = { - nameServerShard.readOperation(_.listUpwardLinks(id)) + nameServerShard.read.any(_.listUpwardLinks(id)) } @throws(classOf[shards.ShardException]) def listDownwardLinks(id: ShardId) = { - nameServerShard.readOperation(_.listDownwardLinks(id)) + nameServerShard.read.any(_.listDownwardLinks(id)) } @throws(classOf[shards.ShardException]) def listLinks() = { - nameServerShard.readOperation(_.listLinks()) + nameServerShard.read.any(_.listLinks()) } @throws(classOf[shards.ShardException]) - def markShardBusy(id: ShardId, busy: Busy.Value) = { - nameServerShard.writeOperation(_.markShardBusy(id, busy)) + def markShardBusy(id: ShardId, busy: Busy.Value) { + nameServerShard.write.foreach(_.markShardBusy(id, busy)) } @throws(classOf[shards.ShardException]) - def setForwarding(forwarding: Forwarding) = { - nameServerShard.writeOperation(_.setForwarding(forwarding)) + def setForwarding(forwarding: Forwarding) { + nameServerShard.write.foreach(_.setForwarding(forwarding)) } @throws(classOf[shards.ShardException]) - def replaceForwarding(oldId: ShardId, newId: ShardId) = { - nameServerShard.writeOperation(_.replaceForwarding(oldId, newId)) + def replaceForwarding(oldId: ShardId, newId: ShardId) { + nameServerShard.write.foreach(_.replaceForwarding(oldId, newId)) } @throws(classOf[shards.ShardException]) def getForwarding(tableId: Int, baseId: Long) = { - nameServerShard.readOperation(_.getForwarding(tableId, baseId)) + nameServerShard.read.any(_.getForwarding(tableId, baseId)) } @throws(classOf[shards.ShardException]) def getForwardingForShard(id: ShardId) = { - nameServerShard.readOperation(_.getForwardingForShard(id)) + nameServerShard.read.any(_.getForwardingForShard(id)) } @throws(classOf[shards.ShardException]) def getForwardings() = { - nameServerShard.readOperation(_.getForwardings()) + nameServerShard.read.any(_.getForwardings()) } @throws(classOf[shards.ShardException]) def shardsForHostname(hostname: String) = { - nameServerShard.readOperation(_.shardsForHostname(hostname)) + nameServerShard.read.any(_.shardsForHostname(hostname)) } @throws(classOf[shards.ShardException]) def listShards() = { - nameServerShard.readOperation(_.listShards()) + nameServerShard.read.any(_.listShards()) } @throws(classOf[shards.ShardException]) def getBusyShards() = { - nameServerShard.readOperation(_.getBusyShards()) + nameServerShard.read.any(_.getBusyShards()) } @throws(classOf[shards.ShardException]) - def rebuildSchema() = { - nameServerShard.writeOperation(_.rebuildSchema()) + def rebuildSchema() { + nameServerShard.write.foreach(_.rebuildSchema()) } @throws(classOf[shards.ShardException]) - def removeForwarding(f: Forwarding) = { - nameServerShard.writeOperation(_.removeForwarding(f)) + def removeForwarding(f: Forwarding) { + nameServerShard.write.foreach(_.removeForwarding(f)) } @throws(classOf[shards.ShardException]) def listHostnames() = { - nameServerShard.readOperation(_.listHostnames()) + nameServerShard.read.any(_.listHostnames()) } @throws(classOf[shards.ShardException]) def listTables() = { - nameServerShard.readOperation(_.listTables()) + nameServerShard.read.any(_.listTables()) } // Remote Host Management @throws(classOf[shards.ShardException]) - def addRemoteHost(h: Host) = { - nameServerShard.writeOperation(_.addRemoteHost(h)) + def addRemoteHost(h: Host) { + nameServerShard.write.foreach(_.addRemoteHost(h)) } @throws(classOf[shards.ShardException]) - def removeRemoteHost(h: String, p: Int) = { - nameServerShard.writeOperation(_.removeRemoteHost(h, p)) + def removeRemoteHost(h: String, p: Int) { + nameServerShard.write.foreach(_.removeRemoteHost(h, p)) } @throws(classOf[shards.ShardException]) - def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) = { - nameServerShard.writeOperation(_.setRemoteHostStatus(h, p, s)) + def setRemoteHostStatus(h: String, p: Int, s: HostStatus.Value) { + nameServerShard.write.foreach(_.setRemoteHostStatus(h, p, s)) } @throws(classOf[shards.ShardException]) - def setRemoteClusterStatus(c: String, s: HostStatus.Value) = { - nameServerShard.writeOperation(_.setRemoteClusterStatus(c, s)) + def setRemoteClusterStatus(c: String, s: HostStatus.Value) { + nameServerShard.write.foreach(_.setRemoteClusterStatus(c, s)) } @throws(classOf[shards.ShardException]) def getRemoteHost(h: String, p: Int) = { - nameServerShard.readOperation(_.getRemoteHost(h, p)) + nameServerShard.read.any(_.getRemoteHost(h, p)) } @throws(classOf[shards.ShardException]) def listRemoteClusters() = { - nameServerShard.readOperation(_.listRemoteClusters()) + nameServerShard.read.any(_.listRemoteClusters()) } @throws(classOf[shards.ShardException]) def listRemoteHosts() = { - nameServerShard.readOperation(_.listRemoteHosts()) + nameServerShard.read.any(_.listRemoteHosts()) } @throws(classOf[shards.ShardException]) def listRemoteHostsInCluster(c: String) = { - nameServerShard.readOperation(_.listRemoteHostsInCluster(c)) + nameServerShard.read.any(_.listRemoteHostsInCluster(c)) } } diff --git a/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala b/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala index bbf4bdd7..05f14f02 100644 --- a/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala +++ b/src/test/scala/com/twitter/gizzard/fake/ReadWriteShardAdapter.scala @@ -4,6 +4,6 @@ package fake import com.twitter.gizzard.shards.RoutingNode class ReadWriteShardAdapter(node: RoutingNode[Shard]) extends Shard { - def get(k: String) = node.readOperation(_.get(k)) - def put(k: String, v: String) = node.writeOperation(_.put(k, v)) + def get(k: String) = node.read.any(_.get(k)) + def put(k: String, v: String) = node.write.map(_.put(k, v)).head } diff --git a/src/test/scala/com/twitter/gizzard/fake/Shard.scala b/src/test/scala/com/twitter/gizzard/fake/Shard.scala index c8b783ea..104b9c6f 100644 --- a/src/test/scala/com/twitter/gizzard/fake/Shard.scala +++ b/src/test/scala/com/twitter/gizzard/fake/Shard.scala @@ -3,5 +3,5 @@ package fake trait Shard extends { @throws(classOf[Throwable]) def get(k: String): Option[String] - @throws(classOf[Throwable]) def put(k: String, v: String):String + @throws(classOf[Throwable]) def put(k: String, v: String): String } diff --git a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala index b7eda9d1..2a1b1c35 100644 --- a/src/test/scala/com/twitter/gizzard/integration/TestServer.scala +++ b/src/test/scala/com/twitter/gizzard/integration/TestServer.scala @@ -165,10 +165,10 @@ trait TestShard { } class TestShardAdapter(s: shards.RoutingNode[TestShard]) extends TestShard { - def put(k: Int, v: String) = s.writeOperation(_.put(k,v)) - def putAll(kvs: Seq[(Int,String)]) = s.writeOperation(_.putAll(kvs)) - def get(k: Int) = s.readOperation(_.get(k)) - def getAll(k:Int, c: Int) = s.readOperation(_.getAll(k,c)) + def put(k: Int, v: String) = s.write.foreach(_.put(k,v)) + def putAll(kvs: Seq[(Int,String)]) = s.write.foreach(_.putAll(kvs)) + def get(k: Int) = s.read.any(_.get(k)) + def getAll(k:Int, c: Int) = s.read.any(_.getAll(k,c)) } class SqlShardFactory(qeFactory: QueryEvaluatorFactory, conn: Connection) extends shards.ShardFactory[TestShard] { diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index f63c1ae7..6770de96 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -5,6 +5,7 @@ import com.twitter.conversions.time._ import org.specs.Specification import org.specs.mock.JMocker import com.twitter.gizzard.nameserver.LoadBalancer +import com.twitter.util.{Return, Throw} object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { @@ -30,7 +31,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name").willReturn(Some("bob")) } - replicatingShard.skip(ShardId("fake", "shard1")).readOperation(_.get("name")) mustEqual Some("bob") + replicatingShard.skip(ShardId("fake", "shard1")).read.any(_.get("name")) mustEqual Some("bob") } "read failover" in { @@ -41,7 +42,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).get("name").willThrow(exception) then one(shard2).get("name").willReturn(Some("bob")) } - replicatingShard.readOperation(_.get("name")) mustEqual Some("bob") + replicatingShard.read.any(_.get("name")) mustEqual Some("bob") } "when all shards throw an exception" in { @@ -51,7 +52,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).get("name") willThrow exception one(shard2).get("name") willThrow exception } - replicatingShard.readOperation(_.get("name")) must throwA[ShardException] + replicatingShard.read.any(_.get("name")) must throwA[ShardException] } } @@ -59,7 +60,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { expect { one(shard1).get("name").willReturn(Some("ted")) } - replicatingShard.readOperation(_.get("name")) mustEqual Some("ted") + replicatingShard.read.any(_.get("name")) mustEqual Some("ted") } "read all shards" in { @@ -69,7 +70,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willReturn Some("bob") } - replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Right(Some("joe")), Right(Some("bob")))) + replicatingShard.read.all(_.get("name")) must haveTheSameElementsAs(List(Return(Some("joe")), Return(Some("bob")))) } "when one fails" in { @@ -80,7 +81,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willReturn Some("bob") } - replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Left(ex), Right(Some("bob")))) + replicatingShard.read.all(_.get("name")) must haveTheSameElementsAs(List(Throw(ex), Return(Some("bob")))) } "when all fail" in { @@ -92,7 +93,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard2).get("name") willThrow ex2 } - replicatingShard.readAllOperation(_.get("name")) must haveTheSameElementsAs(List(Left(ex1), Left(ex2))) + replicatingShard.read.all(_.get("name")) must haveTheSameElementsAs(List(Throw(ex1), Throw(ex2))) } } @@ -103,7 +104,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") one(shard2).put("name", "alice") } - replicatingShard.writeOperation(_.put("name", "alice")) + replicatingShard.write.foreach(_.put("name", "alice")) } "when the first one fails" in { @@ -111,23 +112,29 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") willThrow new ShardException("o noes") one(shard2).put("name", "alice") } - replicatingShard.writeOperation(_.put("name", "alice")) must throwA[Exception] + replicatingShard.write.foreach(_.put("name", "alice")) must throwA[Exception] } "when one replica is black holed" in { expect { + never(shard1).put("name", "alice") one(shard2).put("name", "alice") } val ss = List(blackhole(node1), node2) val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) - holed.writeOperation(_.put("name", "alice")) + holed.write.foreach(_.put("name", "alice")) } "when all replicas are black holed" in { + expect { + never(shard1).put("name", "alice") + never(shard2).put("name", "alice") + } + val ss = shards.map(blackhole) val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) - holed.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] + holed.write.foreach(_.put("name", "alice")) } } @@ -139,7 +146,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "carol") one(shard2).put("name", "carol") } - replicatingShard.writeOperation(_.put("name", "carol")) + replicatingShard.write.foreach(_.put("name", "carol")) } "with an exception" in { @@ -147,23 +154,29 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "carol") willThrow new ShardException("o noes") one(shard2).put("name", "carol") } - replicatingShard.writeOperation(_.put("name", "carol")) must throwA[ShardException] + replicatingShard.write.foreach(_.put("name", "carol")) must throwA[ShardException] } "when one replica is black holed" in { expect { + never(shard1).put("name", "alice") one(shard2).put("name", "alice") } val ss = List(blackhole(node1), node2) val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) - holed.writeOperation(_.put("name", "alice")) + holed.write.foreach(_.put("name", "alice")) } "with all black holes" in { + expect { + never(shard1).put("name", "alice") + never(shard2).put("name", "alice") + } + val ss = shards.map(blackhole) val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) - holed.writeOperation(_.put("name", "alice")) must throwA[ShardBlackHoleException] + holed.write.foreach(_.put("name", "alice")) } } } From 7d6a20d440075cd64ec963f7aa525cd2463490bb Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 16:34:08 -0700 Subject: [PATCH 10/16] add SlaveShard, fix ReplicatingShard construction --- .../twitter/gizzard/config/NameServer.scala | 3 +- .../gizzard/nameserver/ShardRepository.scala | 4 +-- .../gizzard/shards/FailingOverShard.scala | 30 ---------------- .../com/twitter/gizzard/shards/NodeSet.scala | 4 ++- .../gizzard/shards/ReplicatingShard.scala | 34 +++---------------- .../gizzard/shards/WrapperRoutingNodes.scala | 11 ++++-- .../nameserver/ShardRepositorySpec.scala | 19 +++++++---- .../gizzard/shards/ReplicatingShardSpec.scala | 11 +++--- 8 files changed, 35 insertions(+), 81 deletions(-) delete mode 100644 src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala diff --git a/src/main/scala/com/twitter/gizzard/config/NameServer.scala b/src/main/scala/com/twitter/gizzard/config/NameServer.scala index b09fc53d..1a9fbed5 100644 --- a/src/main/scala/com/twitter/gizzard/config/NameServer.scala +++ b/src/main/scala/com/twitter/gizzard/config/NameServer.scala @@ -62,8 +62,7 @@ trait NameServer { val replicaNodes = replicas map { replica => new shards.LeafRoutingNode(replica(), 1) } val shardInfo = new shards.ShardInfo("com.twitter.gizzard.nameserver.ReplicatingShard", "", "") - val loadBalancer = new nameserver.LoadBalancer(replicaNodes) - val replicating = new shards.ReplicatingShard(shardInfo, 0, replicaNodes, loadBalancer, None) + val replicating = new shards.ReplicatingShard(shardInfo, 0, replicaNodes) new nameserver.NameServer(replicating, shardRepository, jobRelay(), getMappingFunction()) } diff --git a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala index dfb415fa..bdc2dc40 100644 --- a/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala +++ b/src/main/scala/com/twitter/gizzard/nameserver/ShardRepository.scala @@ -56,7 +56,7 @@ extends ShardRepository[T] { addRoutingNode(prefix + "BlockedShard", BlockedShard[T] _) addRoutingNode(prefix + "WriteOnlyShard", WriteOnlyShard[T] _) addRoutingNode(prefix + "BlackHoleShard", BlackHoleShard[T] _) - addRoutingNode(prefix + "ReplicatingShard", new shards.ReplicatingShardFactory[T](replicationFuture)) - addRoutingNode(prefix + "FailingOverShard", new shards.FailingOverShardFactory[T](replicationFuture)) + addRoutingNode(prefix + "SlaveShard", SlaveShard[T] _) + addRoutingNode(prefix + "ReplicatingShard", ReplicatingShard[T] _) } } diff --git a/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala b/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala deleted file mode 100644 index dcd16163..00000000 --- a/src/main/scala/com/twitter/gizzard/shards/FailingOverShard.scala +++ /dev/null @@ -1,30 +0,0 @@ -package com.twitter.gizzard -package shards - -import com.twitter.util.Duration -import com.twitter.gizzard.nameserver.FailingOverLoadBalancer - -/* - * The FailingOverLoadBalancer splits the shard list into online and offline shards. - * Online shards are randomized according to the current weight-based formula. Offline - * shards are randomly shuffled. To create the final list, 99% of the time, the first - * online shard is first, followed by all offline shards, followed by the rest of the - * online shards. 1% of the time, the offline shards are all first, to keep them warm. - - * This shard type helps with replication level 3 with one replica designated as a fallback. - * The two online replicas will handle the majority of normal read traffic, but read - * retries will fall over to the offline replica rather than the other online one. - */ - - -class FailingOverShardFactory[T](future: Option[Future]) extends RoutingNodeFactory[T] { - def instantiate(info: shards.ShardInfo, weight: Int, replicas: Seq[RoutingNode[T]]) = { - new ReplicatingShard( - info, - weight, - replicas, - new FailingOverLoadBalancer(replicas), - future - ) - } -} diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala index da02e387..775fc2c3 100644 --- a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -96,8 +96,10 @@ extends NodeIterable[T] { new NodeSet(rootInfo, activeFiltered, blockedFiltered) } + def filterNot(f: (ShardInfo, Option[T]) => Boolean) = filter { (i, s) => !f(i, s) } + def skip(ss: ShardId*) = { val set = ss.toSet - filter { (info, _) => set contains info.id } + filterNot { (info, _) => set contains info.id } } } diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index c15cd565..5c8df77c 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -1,39 +1,15 @@ -package com.twitter.gizzard -package shards +package com.twitter.gizzard.shards -import java.lang.reflect.UndeclaredThrowableException -import java.sql.SQLException -import java.util.Random -import java.util.concurrent.{ExecutionException, TimeoutException, TimeUnit} -import scala.collection.mutable -import scala.util.Sorting import com.twitter.gizzard.nameserver.LoadBalancer -import com.twitter.gizzard.thrift.conversions.Sequences._ -import com.twitter.util.Duration -import com.twitter.logging.Logger -class ReplicatingShardFactory[T](future: Option[Future]) extends RoutingNodeFactory[T] { - def instantiate(shardInfo: shards.ShardInfo, weight: Int, replicas: Seq[RoutingNode[T]]) = { - new ReplicatingShard( - shardInfo, - weight, - replicas, - new LoadBalancer(replicas), - future - ) - } -} - -class ReplicatingShard[T]( +case class ReplicatingShard[T]( val shardInfo: ShardInfo, val weight: Int, - val children: Seq[RoutingNode[T]], - val loadBalancer: (() => Seq[RoutingNode[T]]), - val future: Option[Future]) + val children: Seq[RoutingNode[T]]) extends RoutingNode[T] { - import RoutingNode._ + protected def loadBalancer = new LoadBalancer(children) protected[shards] def collectedShards = loadBalancer() flatMap { _.collectedShards } @@ -46,7 +22,7 @@ extends RoutingNode[T] { } else if (filtered.size == children.size) { this } else { - new ReplicatingShard[T](shardInfo, weight, filtered, new LoadBalancer(filtered), future) + new ReplicatingShard[T](shardInfo, weight, filtered) } } } diff --git a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala index 82334867..103d8bc9 100644 --- a/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala +++ b/src/main/scala/com/twitter/gizzard/shards/WrapperRoutingNodes.scala @@ -10,6 +10,7 @@ abstract class WrapperRoutingNode[T] extends RoutingNode[T] { protected[shards] def collectedShards = children flatMap { _.collectedShards.map(leafTransform) } } + // BlockedShard. Refuse and fail all traffic. case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { @@ -17,7 +18,6 @@ case class BlockedShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Rout } - // BlackHoleShard. Silently refuse all traffic. case class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { @@ -25,7 +25,6 @@ case class BlackHoleShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Ro } - // WriteOnlyShard. Fail all read traffic. case class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { @@ -33,9 +32,15 @@ case class WriteOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[Ro } - // ReadOnlyShard. Fail all write traffic. case class ReadOnlyShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(writeBehavior = RoutingNode.Deny) } + + +// SlaveShard. Silently refuse all write traffic. + +case class SlaveShard[T](shardInfo: ShardInfo, weight: Int, children: Seq[RoutingNode[T]]) extends WrapperRoutingNode[T] { + protected def leafTransform(l: RoutingNode.Leaf[T]) = l.copy(writeBehavior = RoutingNode.Ignore) +} diff --git a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala index c7e0a330..1fda44f8 100644 --- a/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala +++ b/src/test/scala/com/twitter/gizzard/nameserver/ShardRepositorySpec.scala @@ -30,15 +30,20 @@ object ShardRepositorySpec extends ConfiguredSpecification with JMocker with Cla repository.factory("com.twitter.gizzard.shards.BlockedShard").instantiate(info, 1, shard) must haveClass[shards.BlockedShard[AnyRef]] repository.factory("com.example.bogis.BlockedShard") must throwA[NoSuchElementException] } + "find a blackhole shard" in { + repository.factory("BlackHoleShard").instantiate(info, 1, shard) must haveClass[shards.BlackHoleShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.BlackHoleShard").instantiate(info, 1, shard) must haveClass[shards.BlackHoleShard[AnyRef]] + repository.factory("com.example.bogis.BlackHoleShard") must throwA[NoSuchElementException] + } + "find a slave shard" in { + repository.factory("SlaveShard").instantiate(info, 1, shard) must haveClass[shards.SlaveShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.SlaveShard").instantiate(info, 1, shard) must haveClass[shards.SlaveShard[AnyRef]] + repository.factory("com.example.bogis.SlaveShard") must throwA[NoSuchElementException] + } "find a replicating shard" in { - repository.factory("ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[AnyRef]] - repository.factory("com.twitter.gizzard.shards.ReplicatingShard") must haveClass[shards.ReplicatingShardFactory[AnyRef]] + repository.factory("ReplicatingShard").instantiate(info, 1, shard) must haveClass[shards.ReplicatingShard[AnyRef]] + repository.factory("com.twitter.gizzard.shards.ReplicatingShard").instantiate(info, 1, shard) must haveClass[shards.ReplicatingShard[AnyRef]] repository.factory("com.example.bogis.ReplicatingShard") must throwA[NoSuchElementException] } - "find a failing over shard" in { - repository.factory("FailingOverShard") must haveClass[shards.FailingOverShardFactory[AnyRef]] - repository.factory("com.twitter.gizzard.shards.FailingOverShard") must haveClass[shards.FailingOverShardFactory[AnyRef]] - repository.factory("com.example.bogis.FailingOverShard") must throwA[NoSuchElementException] - } } } diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 6770de96..51bd91d9 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -20,11 +20,10 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { new LeafRoutingNode(s, new ShardInfo("", "shard"+ (i + 1), "fake"), 1) } - val future = new Future("Future!", 1, 1, 1.second, 1.second) val shards = List(node1, node2) val replicatingShardInfo = new ShardInfo("", "replicating_shard", "hostname") - var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, () => shards, Some(future)) + var replicatingShard = ReplicatingShard(replicatingShardInfo, 1, shards) "filters shards" in { expect { @@ -139,8 +138,6 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { } "in series" in { - var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards, () => shards, None) - "normal" in { expect { one(shard1).put("name", "carol") @@ -164,7 +161,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { } val ss = List(blackhole(node1), node2) - val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) + val holed = ReplicatingShard(replicatingShardInfo, 1, ss) holed.write.foreach(_.put("name", "alice")) } @@ -175,7 +172,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { } val ss = shards.map(blackhole) - val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, None) + val holed = ReplicatingShard(replicatingShardInfo, 1, ss) holed.write.foreach(_.put("name", "alice")) } } @@ -192,7 +189,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val mock2 = mock[EnufShard] val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(_, 1)) val shards = List(node1, node2) - val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards, () => shards, Some(future)) + val shard = ReplicatingShard[EnufShard](shardInfo, 1, shards) "first shard has data" in { expect { From 63275b417342b98c42668ee6ea9fd180f2b58238 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 16:34:34 -0700 Subject: [PATCH 11/16] replace parallel fanout with `par` chain operation --- .../com/twitter/gizzard/shards/NodeSet.scala | 4 ++ .../twitter/gizzard/shards/ParNodeSet.scala | 37 +++++++++++++++++++ .../gizzard/shards/ReplicatingShardSpec.scala | 12 +++--- 3 files changed, 47 insertions(+), 6 deletions(-) create mode 100644 src/main/scala/com/twitter/gizzard/shards/ParNodeSet.scala diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala index 775fc2c3..77230c2f 100644 --- a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -90,6 +90,10 @@ class NodeSet[T]( val blockedShards: Seq[ShardInfo]) extends NodeIterable[T] { + def par(implicit cfg: ParConfig = ParConfig.default): NodeSet[T] = { + new ParNodeSet(rootInfo, activeShards, blockedShards, cfg.pool, cfg.timeout) + } + def filter(f: (ShardInfo, Option[T]) => Boolean) = { val activeFiltered = activeShards filter { case (i, s) => f(i, Some(s)) } val blockedFiltered = blockedShards filter { i => f(i, None) } diff --git a/src/main/scala/com/twitter/gizzard/shards/ParNodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/ParNodeSet.scala new file mode 100644 index 00000000..3fe4ab05 --- /dev/null +++ b/src/main/scala/com/twitter/gizzard/shards/ParNodeSet.scala @@ -0,0 +1,37 @@ +package com.twitter.gizzard.shards + +import java.util.concurrent.Executors +import scala.collection.generic.CanBuild +import com.twitter.util.Duration +import com.twitter.util.FuturePool +import com.twitter.util.{Try, Throw} +import com.twitter.conversions.time._ + +class ParConfig(val pool: FuturePool, val timeout: Duration) + +object ParConfig { + lazy val default = new ParConfig(FuturePool(Executors.newCachedThreadPool()), 99.days) +} + +trait ParNodeIterable[T] extends NodeIterable[T] { + protected def futurePool: FuturePool + protected def timeout: Duration + + override def all[R, That](f: T => R)(implicit bf: CanBuild[Try[R], That] = Seq.canBuildFrom[Try[R]]): That = { + val futures = activeShards map { case (_, s) => futurePool(f(s)) } + + val b = bf() + for (future <- futures) b += Try(future.get(timeout).apply()) + for (s <- blockedShards) b += Throw(new ShardOfflineException(s.id)) + b.result + } +} + +class ParNodeSet[T]( + root: ShardInfo, + active: Seq[(ShardInfo, T)], + blocked: Seq[ShardInfo], + protected val futurePool: FuturePool, + protected val timeout: Duration) +extends NodeSet[T](root, active, blocked) +with ParNodeIterable[T] diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 51bd91d9..11fa8151 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -103,7 +103,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") one(shard2).put("name", "alice") } - replicatingShard.write.foreach(_.put("name", "alice")) + replicatingShard.write.par.foreach(_.put("name", "alice")) } "when the first one fails" in { @@ -111,7 +111,7 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { one(shard1).put("name", "alice") willThrow new ShardException("o noes") one(shard2).put("name", "alice") } - replicatingShard.write.foreach(_.put("name", "alice")) must throwA[Exception] + replicatingShard.write.par.foreach(_.put("name", "alice")) must throwA[Exception] } "when one replica is black holed" in { @@ -121,8 +121,8 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { } val ss = List(blackhole(node1), node2) - val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) - holed.write.foreach(_.put("name", "alice")) + val holed = ReplicatingShard(replicatingShardInfo, 1, ss) + holed.write.par.foreach(_.put("name", "alice")) } "when all replicas are black holed" in { @@ -132,8 +132,8 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { } val ss = shards.map(blackhole) - val holed = new ReplicatingShard(replicatingShardInfo, 1, ss, () => ss, Some(future)) - holed.write.foreach(_.put("name", "alice")) + val holed = ReplicatingShard(replicatingShardInfo, 1, ss) + holed.write.par.foreach(_.put("name", "alice")) } } From cda3958c85b221c70cb8ded4f24f4658d0fa86c1 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 21:12:12 -0700 Subject: [PATCH 12/16] tweak loadBalancer to allow override in tests --- .../com/twitter/gizzard/shards/ReplicatingShard.scala | 2 +- .../com/twitter/gizzard/shards/ReplicatingShardSpec.scala | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala index 5c8df77c..2a422921 100644 --- a/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala +++ b/src/main/scala/com/twitter/gizzard/shards/ReplicatingShard.scala @@ -9,7 +9,7 @@ case class ReplicatingShard[T]( val children: Seq[RoutingNode[T]]) extends RoutingNode[T] { - protected def loadBalancer = new LoadBalancer(children) + protected def loadBalancer() = new LoadBalancer(children).apply() protected[shards] def collectedShards = loadBalancer() flatMap { _.collectedShards } diff --git a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala index 11fa8151..b0e5c3bd 100644 --- a/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala +++ b/src/test/scala/com/twitter/gizzard/shards/ReplicatingShardSpec.scala @@ -23,7 +23,9 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val shards = List(node1, node2) val replicatingShardInfo = new ShardInfo("", "replicating_shard", "hostname") - var replicatingShard = ReplicatingShard(replicatingShardInfo, 1, shards) + var replicatingShard = new ReplicatingShard(replicatingShardInfo, 1, shards) { + override protected def loadBalancer() = children.toList + } "filters shards" in { expect { @@ -189,7 +191,9 @@ object ReplicatingShardSpec extends ConfiguredSpecification with JMocker { val mock2 = mock[EnufShard] val List(node1, node2) = List(mock1, mock2).map(new LeafRoutingNode(_, 1)) val shards = List(node1, node2) - val shard = ReplicatingShard[EnufShard](shardInfo, 1, shards) + val shard = new ReplicatingShard[EnufShard](shardInfo, 1, shards) { + override protected def loadBalancer() = children.toList + } "first shard has data" in { expect { From 4d7e17b0f736a9ac6448e9ba8f8f682952298df3 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 22:01:25 -0700 Subject: [PATCH 13/16] add fmap for future api support --- .../scala/com/twitter/gizzard/shards/NodeSet.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala index 77230c2f..b17ccf15 100644 --- a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -1,7 +1,7 @@ package com.twitter.gizzard.shards import scala.collection.generic.CanBuild -import com.twitter.util.{Try, Throw} +import com.twitter.util.{Try, Throw, Future} // For read or write, three node states: // - normal: should apply normally @@ -52,6 +52,14 @@ trait NodeIterable[T] { throw new ShardOfflineException(rootInfo.id) } + // XXX: it would be nice to have a way to implement all in terms of fmap. :( + def fmap[R, That](f: T => Future[R])(implicit bf: CanBuild[Future[R], That] = Seq.canBuildFrom[Future[R]]): That = { + val b = bf() + for ((i, s) <- activeShards) b += f(s) + for (s <- blockedShards) b += Future.exception(new ShardOfflineException(s.id)) + b.result + } + def all[R, That](f: T => R)(implicit bf: CanBuild[Try[R], That] = Seq.canBuildFrom[Try[R]]): That = { val b = bf() for ((i, s) <- activeShards) b += Try(f(s)) From 786646d662b8cdb933f0f0f758896b44ce6ac2e1 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Thu, 2 Jun 2011 22:07:20 -0700 Subject: [PATCH 14/16] anyOption --- .../scala/com/twitter/gizzard/shards/NodeSet.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala index b17ccf15..686c9456 100644 --- a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -32,24 +32,26 @@ trait NodeIterable[T] { def containsBlocked = !blockedShards.isEmpty + def anyOption[R](f: T => R): Option[R] = _any(f, iterator) + def any[R](f: T => R): R = { if (activeShards.isEmpty && blockedShards.isEmpty) { throw new ShardBlackHoleException(rootInfo.id) } - _any(f, iterator) + _any(f, iterator).getOrElse(throw new ShardOfflineException(rootInfo.id)) } - protected final def _any[R](f: T => R, iter: Iterator[T]): R = { + protected final def _any[R](f: T => R, iter: Iterator[T]): Option[R] = { while (iter.hasNext) { try { - return f(iter.next) + return Some(f(iter.next)) } catch { case e: ShardException => () } } - throw new ShardOfflineException(rootInfo.id) + None } // XXX: it would be nice to have a way to implement all in terms of fmap. :( From 76fd7cd72de60bec0711f71dd9fa3bfcd0819432 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Fri, 3 Jun 2011 13:55:15 -0700 Subject: [PATCH 15/16] after waffling, support functions that return Try as well --- .../com/twitter/gizzard/shards/NodeSet.scala | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala index 686c9456..66464474 100644 --- a/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala +++ b/src/main/scala/com/twitter/gizzard/shards/NodeSet.scala @@ -1,5 +1,6 @@ package com.twitter.gizzard.shards +import scala.annotation.tailrec import scala.collection.generic.CanBuild import com.twitter.util.{Try, Throw, Future} @@ -32,26 +33,27 @@ trait NodeIterable[T] { def containsBlocked = !blockedShards.isEmpty - def anyOption[R](f: T => R): Option[R] = _any(f, iterator) + def anyOption[R](f: T => R): Option[R] = _any(iterator, s => Try(f(s)) ).toOption + + def tryAny[R](f: T => Try[R]): Try[R] = _any(iterator, f) def any[R](f: T => R): R = { if (activeShards.isEmpty && blockedShards.isEmpty) { throw new ShardBlackHoleException(rootInfo.id) } - _any(f, iterator).getOrElse(throw new ShardOfflineException(rootInfo.id)) + _any(iterator, s => Try(f(s)) ).apply() } - protected final def _any[R](f: T => R, iter: Iterator[T]): Option[R] = { - while (iter.hasNext) { - try { - return Some(f(iter.next)) - } catch { - case e: ShardException => () + @tailrec protected final def _any[R](iter: Iterator[T], f: T => Try[R]): Try[R] = { + if (iter.hasNext) { + f(iter.next) match { + case rv if rv.isReturn => rv + case _ => _any(iter, f) } + } else { + Throw(new ShardOfflineException(rootInfo.id)) } - - None } // XXX: it would be nice to have a way to implement all in terms of fmap. :( @@ -63,8 +65,12 @@ trait NodeIterable[T] { } def all[R, That](f: T => R)(implicit bf: CanBuild[Try[R], That] = Seq.canBuildFrom[Try[R]]): That = { + tryAll { s => Try(f(s)) } + } + + def tryAll[R, That](f: T => Try[R])(implicit bf: CanBuild[Try[R], That] = Seq.canBuildFrom[Try[R]]): That = { val b = bf() - for ((i, s) <- activeShards) b += Try(f(s)) + for ((i, s) <- activeShards) b += f(s) for (s <- blockedShards) b += Throw(new ShardOfflineException(s.id)) b.result } From 94057db6d64b93722fb6c985570d924cf1d6e578 Mon Sep 17 00:00:00 2001 From: Matt Freels Date: Fri, 3 Jun 2011 15:56:57 -0700 Subject: [PATCH 16/16] add back logging to RoutingNode's deprecated handlers --- .../twitter/gizzard/shards/RoutingNode.scala | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala index 9ee04567..13f03531 100644 --- a/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala +++ b/src/main/scala/com/twitter/gizzard/shards/RoutingNode.scala @@ -3,6 +3,7 @@ package com.twitter.gizzard.shards import java.lang.reflect.UndeclaredThrowableException import java.util.concurrent.{ExecutionException, TimeoutException} import com.twitter.util.{Try, Return, Throw} +import com.twitter.logging.Logger abstract class RoutingNodeFactory[T] { @@ -32,6 +33,8 @@ abstract class RoutingNode[T] { import RoutingNode._ + protected val log = Logger.get + protected[shards] def collectedShards: Seq[Leaf[T]] protected def nodeSetFromCollected(filter: Leaf[T] => Behavior) = { @@ -52,13 +55,15 @@ abstract class RoutingNode[T] { } @deprecated("use read.all instead") - def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] = read.all(f) map { + def readAllOperation[A](f: T => A): Seq[Either[Throwable,A]] = read.all(f) map { f => Try(f()) } map { case Return(r) => Right(r) case Throw(e) => Left(e) } @deprecated("use read.any instead") - def readOperation[A](f: T => A) = read.any(f) + def readOperation[A](f: T => A) = read.tryAny { shard => + Try(f(shard)) onFailure { e => logException(e, shard) } + } @deprecated("use write.all instead") def writeOperation[A](f: T => A) = { @@ -88,7 +93,7 @@ abstract class RoutingNode[T] { return result } } catch { - case e: ShardException => () // XXX: log error + case e => logException(e, shard) } } @@ -99,12 +104,18 @@ abstract class RoutingNode[T] { } } - protected def normalizeException(ex: Throwable, shardId: ShardId): Option[Throwable] = ex match { + protected def logException(e: Throwable, shard: T) { + val shardId = (collectedShards find { l => l.shard == shard }).get.info.id + val normalized = normalizeException(e, shardId) + + log.warning(e, "Error on %s: %s", shardId, e) + } + + protected def normalizeException(ex: Throwable, shardId: ShardId): Throwable = ex match { case e: ExecutionException => normalizeException(e.getCause, shardId) // fondly known as JavaOutrageException case e: UndeclaredThrowableException => normalizeException(e.getCause, shardId) - case e: ShardBlackHoleException => None - case e: TimeoutException => Some(new ReplicatingShardTimeoutException(shardId, e)) - case e => Some(e) + case e: TimeoutException => new ReplicatingShardTimeoutException(shardId, e) + case e => e } }