Skip to content
This repository was archived by the owner on Jan 20, 2022. It is now read-only.

Splits the storm test code out into its own package to help end users be... #442

Merged
merged 3 commits into from
Feb 12, 2014
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 16 additions & 0 deletions project/Build.scala
Original file line number Diff line number Diff line change
Expand Up @@ -220,6 +220,22 @@ object SummingbirdBuild extends Build {
summingbirdBatch
)

lazy val summingbirdStormTest = module("storm-tests").settings(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

minor, can we call this storm-test?

parallelExecution in Test := false,
libraryDependencies ++= Seq(
"com.twitter" %% "algebird-core" % algebirdVersion,
"com.twitter" %% "bijection-core" % bijectionVersion,
"com.twitter" %% "storehaus-core" % storehausVersion,
"com.twitter" %% "storehaus-algebra" % storehausVersion,
"com.twitter" %% "tormenta-core" % tormentaVersion,
withCross("com.twitter" %% "util-core" % utilVersion),
"storm" % "storm" % "0.9.0-wip15" % "provided"
)
).dependsOn(
summingbirdCore % "test->test;compile->compile",
summingbirdStorm
)

lazy val summingbirdScalding = module("scalding").settings(
libraryDependencies ++= Seq(
"com.backtype" % "dfs-datastores" % dfsDatastoresVersion,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
Copyright 2013 Twitter, Inc.

Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package com.twitter.summingbird.storm

import com.twitter.algebird.Semigroup
import backtype.storm.{ Config => BacktypeStormConfig, LocalCluster, Testing }
import backtype.storm.testing.{ CompleteTopologyParam, MockedSources }
import com.twitter.summingbird.storm.spout.TraversableSpout
import com.twitter.summingbird._
import com.twitter.summingbird.planner._
import com.twitter.tormenta.spout.Spout
import scala.collection.JavaConverters._
import java.security.Permission


/**
* This stops Storm's exception handling triggering an exit(1)
*/
private[storm] class MySecurityManager extends SecurityManager {
override def checkExit(status: Int): Unit = {
throw new SecurityException();
}
override def checkAccess(t: Thread) = {}
override def checkPermission(p: Permission) = {}
}

/*
* This is a wrapper to run a storm topology.
* We use the SecurityManager code to catch the System.exit storm calls when it
* fails. We wrap it into a normal exception instead so it can report better/retry.
*/

object StormTestRun {
private def completeTopologyParam(conf: BacktypeStormConfig) = {
val ret = new CompleteTopologyParam()
ret.setMockedSources(new MockedSources)
ret.setStormConf(conf)
ret.setCleanupState(false)
ret
}


private def tryRun(plannedTopology: PlannedTopology): Unit = {
//Before running the external Command
val oldSecManager = System.getSecurityManager()
System.setSecurityManager(new MySecurityManager());
try {
val cluster = new LocalCluster()
Testing.completeTopology(cluster, plannedTopology.topology, completeTopologyParam(plannedTopology.config))
// Sleep to prevent this race: https://github.com/nathanmarz/storm/pull/667
Thread.sleep(1000)
cluster.shutdown
} finally {
System.setSecurityManager(oldSecManager)
}
}

def apply(graph: TailProducer[Storm, Any])(implicit storm: Storm) {
val topo = storm.plan(graph)
apply(topo)
}

def simpleRun[T, K, V: Semigroup](original: List[T], mkJob: (Producer[Storm, T], Storm#Store[K, V]) => TailProducer[Storm, Any])
: TestStore[K, V] = {

implicit def extractor[T]: TimeExtractor[T] = TimeExtractor(_ => 0L)

val (id, store) = TestStore.createStore[K, V]()

val job = mkJob(
Storm.source(TraversableSpout(original)),
store
)

implicit val s = Storm.local(Map())
apply(job)
TestStore[K, V](id).getOrElse(sys.error("Error running test, unable to find store at the end"))
}

def apply(plannedTopology: PlannedTopology) {
this.synchronized {
try {
tryRun(plannedTopology)
} catch {
case _: Throwable =>
Thread.sleep(3000)
tryRun(plannedTopology)
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
Copyright 2013 Twitter, Inc.

Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package com.twitter.summingbird.storm

import com.twitter.algebird.Semigroup
import com.twitter.storehaus.algebra.MergeableStore
import com.twitter.summingbird.batch.{BatchID, Batcher}
import com.twitter.util.Future
import java.util.{Collections, HashMap, Map => JMap, UUID}
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.SynchronizedMap
import java.util.WeakHashMap
import scala.collection.JavaConverters._



object TestStore {
private val testStores = new WeakHashMap[String, TestStore[_, _]]

def apply[K, V: Semigroup](storeID: String): Option[TestStore[K, V]] =
(Option(testStores.get(storeID)).map{s =>
s.asInstanceOf[TestStore[K, V]]})

private def buildStore[K, V: Semigroup](initialData: Map[K, V]) : String = {
val storeID = UUID.randomUUID.toString
val newInitStore = TestStore[K, V](storeID, initialData)
testStores.synchronized {
testStores.put(storeID, newInitStore)
}
storeID
}

def createBatchedStore[K, V]
(initialData: Map[(K, BatchID), V] = Map.empty[(K, BatchID), V])
(implicit batcher: Batcher, valueSG: Semigroup[V]):
(String, MergeableStoreSupplier[K, V]) = {

val storeID = buildStore[(K, BatchID), V](initialData)
val supplier = MergeableStoreSupplier.from(
TestStore.apply[(K, BatchID), V](storeID)
.getOrElse(sys.error("Weak hash map no longer contains store"))
)
(storeID, supplier)
}

def createStore[K, V: Semigroup](initialData: Map[K, V] = Map.empty[K, V]):
(String, MergeableStoreSupplier[K, V]) = {
val storeID = buildStore[K, V](initialData)
val supplier = MergeableStoreSupplier.fromOnlineOnly(
TestStore.apply[K, V](storeID)
.getOrElse(sys.error("Weak hash map no longer contains store"))
)

(storeID, supplier)
}
}


case class TestStore[K, V: Semigroup](storeID: String, initialData: Map[K, V]) extends MergeableStore[K, V] {
private val backingStore: JMap[K, Option[V]] =
Collections.synchronizedMap(new HashMap[K, Option[V]]())
val updates: AtomicInteger = new AtomicInteger(0)
val reads: AtomicInteger = new AtomicInteger(0)

def toScala: Map[K, V] = backingStore.asScala.collect{ case (k, Some(v)) => (k, v)}.toMap

private def getOpt(k: K) = {
reads.incrementAndGet
Option(backingStore.get(k)).flatMap(i => i)
}

val semigroup = implicitly[Semigroup[V]]

override def get(k: K) = Future.value(getOpt(k))

override def put(pair: (K, Option[V])) = {
val (k, optV) = pair
if (optV.isDefined)
backingStore.put(k, optV)
else
backingStore.remove(k)
updates.incrementAndGet
Future.Unit
}

override def merge(pair: (K, V)) = {
val (k, v) = pair
val oldV = getOpt(k)
val newV = Semigroup.plus(Some(v), oldV)
updates.incrementAndGet
backingStore.put(k, newV)
Future.value(oldV)
}

}
Loading