You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by cr...@apache.org on 2013/08/12 18:21:31 UTC

[01/15] initial import.

Updated Branches:
  refs/heads/master [created] 5ff71e51f


http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala
new file mode 100644
index 0000000..4bb89b5
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.samza.job.StreamJobFactory
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.samza.config.Config
+
+class YarnJobFactory extends StreamJobFactory {
+  def getJob(config: Config) = {
+    // TODO fix this. needed to support http package locations.
+    val hConfig = new YarnConfiguration
+    hConfig.set("fs.http.impl", "samza.util.hadoop.HttpFileSystem")
+
+    new YarnJob(config, hConfig)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala b/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala
new file mode 100644
index 0000000..85813b1
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util.hadoop
+
+import java.io.IOException
+import java.net.URI
+
+import org.apache.commons.httpclient.methods.GetMethod
+import org.apache.commons.httpclient.HttpClient
+import org.apache.commons.httpclient.HttpStatus
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.fs.FSDataInputStream
+import org.apache.hadoop.fs.FSDataOutputStream
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.util.Progressable
+
+import grizzled.slf4j.Logging
+
+class HttpFileSystem extends FileSystem with Logging {
+  val DEFAULT_BLOCK_SIZE = 4 * 1024;
+  var uri: URI = null
+
+  override def initialize(uri: URI, conf: Configuration) {
+    super.initialize(uri, conf)
+    debug("init uri %s" format (uri))
+    this.uri = uri
+  }
+
+  override def getUri = uri
+
+  override def open(f: Path, bufferSize: Int): FSDataInputStream = {
+    debug("open http file %s" format (f))
+    val client = new HttpClient
+    val method = new GetMethod(f.toUri.toString)
+    val statusCode = client.executeMethod(method)
+
+    if (statusCode != HttpStatus.SC_OK) {
+      warn("got status code %d for uri %s" format (statusCode, uri))
+      throw new IOException("Bad status code returned by http server " + f + ": " + statusCode)
+    }
+
+    new FSDataInputStream(new HttpInputStream(method.getResponseBodyAsStream))
+  }
+
+  override def create(f: Path,
+    permission: FsPermission,
+    overwrite: Boolean,
+    bufferSize: Int,
+    replication: Short,
+    blockSize: Long,
+    progress: Progressable): FSDataOutputStream = null
+
+  override def append(f: Path, bufferSize: Int, progress: Progressable): FSDataOutputStream = null
+
+  override def rename(src: Path, dst: Path): Boolean = false
+
+  override def delete(f: Path, recursive: Boolean): Boolean = false
+
+  override def listStatus(f: Path): Array[FileStatus] = null
+
+  override def setWorkingDirectory(newDir: Path) {}
+
+  override def getWorkingDirectory(): Path = new Path("/")
+
+  override def mkdirs(f: Path, permission: FsPermission): Boolean = false
+
+  override def getFileStatus(f: Path): FileStatus = {
+    val length = -1
+    val isDir = false
+    val blockReplication = 1
+    val blockSize = DEFAULT_BLOCK_SIZE
+    val modTime = 0
+    val fs = new FileStatus(length, isDir, blockReplication, blockSize, modTime, f)
+    debug("file status for %s is %s" format (f, fs))
+    return fs
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala b/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala
new file mode 100644
index 0000000..a5458fb
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util.hadoop
+
+import java.io.IOException
+import java.io.InputStream
+
+import org.apache.hadoop.fs.FSInputStream
+
+class HttpInputStream(is: InputStream) extends FSInputStream {
+  val lock: AnyRef = new Object
+  var pos: Long = 0
+
+  override def seek(pos: Long) = throw new IOException("Seek not supported");
+
+  override def getPos: Long = pos
+
+  override def seekToNewSource(targetPos: Long): Boolean = throw new IOException("Seek not supported");
+
+  override def read: Int = {
+    lock.synchronized {
+      var byteRead = is.read()
+      if (byteRead >= 0) {
+        pos += 1
+      }
+      byteRead
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala
new file mode 100644
index 0000000..8fce8a7
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.webapp
+
+import org.scalatra._
+import scalate.ScalateSupport
+import org.apache.samza.config.Config
+import org.apache.samza.job.yarn.SamzaAppMasterState
+import org.apache.samza.job.yarn.ClientHelper
+import org.apache.samza.metrics._
+import scala.collection.JavaConversions._
+import scala.collection.immutable.TreeMap
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.codehaus.jackson.map.ObjectMapper
+import java.util.HashMap
+
+class ApplicationMasterRestServlet(config: Config, state: SamzaAppMasterState, registry: ReadableMetricsRegistry) extends ScalatraServlet with ScalateSupport {
+  val yarnConfig = new YarnConfiguration
+  val client = new ClientHelper(yarnConfig)
+  val jsonMapper = new ObjectMapper()
+
+  before() {
+    contentType = "application/json"
+  }
+
+  get("/metrics") {
+    val metricMap = new HashMap[String, java.util.Map[String, Object]]
+
+    // build metric map
+    registry.getGroups.foreach(group => {
+      val groupMap = new HashMap[String, Object]
+
+      registry.getGroup(group).foreach {
+        case (name, metric) =>
+          metric.visit(new MetricsVisitor() {
+            def counter(counter: Counter) =
+              groupMap.put(counter.getName, counter.getCount: java.lang.Long)
+
+            def gauge[T](gauge: Gauge[T]) =
+              groupMap.put(gauge.getName, gauge.getValue.asInstanceOf[java.lang.Object])
+          })
+      }
+
+      metricMap.put(group, groupMap)
+    })
+
+    jsonMapper.writeValueAsString(metricMap)
+  }
+
+  get("/task-context") {
+    // sick of fighting with scala.. just using java map for now
+    val contextMap = new HashMap[String, Object]
+
+    contextMap.put("task-id", state.taskId: java.lang.Integer)
+    contextMap.put("name", state.containerId.toString)
+
+    jsonMapper.writeValueAsString(contextMap)
+  }
+
+  get("/am") {
+    val containers = new HashMap[String, HashMap[String, Object]]
+
+    state.runningTasks.values.foreach(c => {
+      val containerIdStr = c.getId.toString
+      val containerMap = new HashMap[String, Object]
+      val taskId = state.runningTasks.filter { case (_, container) => container.getId.toString.equals(containerIdStr) }.keys.head
+      var partitions = new java.util.ArrayList(state.taskPartitions.get(taskId).get)
+
+      containerMap.put("yarn-address", c.getNodeHttpAddress)
+      containerMap.put("partitions", partitions)
+      containerMap.put("task-id", taskId.toString)
+      containers.put(containerIdStr, containerMap)
+    })
+
+    val status = Map[String, Object](
+      "app-attempt-id" -> state.appAttemptId.toString,
+      "container-id" -> state.containerId.toString,
+      "containers" -> containers,
+      "host" -> "%s:%s".format(state.nodeHost, state.rpcPort))
+
+    jsonMapper.writeValueAsString(new HashMap[String, Object](status))
+  }
+
+  get("/am/kill") {
+    client.kill(state.appAttemptId.getApplicationId)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala
new file mode 100644
index 0000000..cbd7c1e
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.webapp
+
+import org.scalatra._
+import scalate.ScalateSupport
+import org.apache.samza.job.yarn.SamzaAppMasterState
+import org.apache.samza.config.Config
+import scala.collection.JavaConversions._
+import scala.collection.immutable.TreeMap
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+class ApplicationMasterWebServlet(config: Config, state: SamzaAppMasterState) extends ScalatraServlet with ScalateSupport {
+  val yarnConfig = new YarnConfiguration
+
+  before() {
+    contentType = "text/html"
+  }
+
+  get("/") {
+    layoutTemplate("/WEB-INF/views/index.scaml",
+      "config" -> TreeMap(config.toMap.toArray: _*),
+      "state" -> state,
+      "rmHttpAddress" -> YarnConfiguration.getRMWebAppURL(yarnConfig))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/webapp/WebAppServer.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/WebAppServer.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/WebAppServer.scala
new file mode 100644
index 0000000..bb5c297
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/WebAppServer.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.webapp
+
+import javax.servlet.Servlet
+import org.eclipse.jetty.server.Server
+import org.eclipse.jetty.servlet.{ DefaultServlet, ServletHolder }
+import org.eclipse.jetty.webapp.WebAppContext
+
+class WebAppServer(rootPath: String, port: Int) {
+  val server = new Server(port)
+  val context = new WebAppContext
+
+  // add a default holder to deal with static files
+  val defaultHolder = new ServletHolder(classOf[DefaultServlet])
+  defaultHolder.setName("default")
+  context.setContextPath(rootPath)
+  context.addServlet(defaultHolder, "/css/*")
+  context.addServlet(defaultHolder, "/js/*")
+
+  // TODO This is where you'd add Hadoop's Kerberos security filters.
+  // context.addFilter(classOf[YourApplicationEndpointFilter], "/*", 0)
+
+  def addServlet(subPath: String, servlet: Servlet) {
+    context.addServlet(new ServletHolder(servlet), subPath)
+  }
+
+  def start {
+    context.setContextPath("/");
+    context.setResourceBase(getClass.getClassLoader.getResource("scalate").toExternalForm)
+    server.setHandler(context)
+    server.start
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterLifecycle.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterLifecycle.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterLifecycle.scala
new file mode 100644
index 0000000..8bf48eb
--- /dev/null
+++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterLifecycle.scala
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse
+import org.apache.hadoop.yarn.api.records.ContainerId
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+import org.apache.hadoop.yarn.api.records.Priority
+import org.apache.hadoop.yarn.api.records.Resource
+import org.apache.hadoop.yarn.client.AMRMClient
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.api.records.Resource
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse
+import org.apache.hadoop.yarn.service._
+import org.apache.samza.SamzaException
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType
+
+class TestSamzaAppMasterLifecycle {
+  val amClient = new AMRMClient {
+    var host = ""
+    var port = 0
+    var status: FinalApplicationStatus = null
+    def registerApplicationMaster(appHostName: String, appHostPort: Int, appTrackingUrl: String): RegisterApplicationMasterResponse = {
+      this.host = appHostName
+      this.port = appHostPort
+      new RegisterApplicationMasterResponse {
+        def setApplicationACLs(map: java.util.Map[ApplicationAccessType, String]) = null
+        def getApplicationACLs = null
+        def setMaximumResourceCapability(r: Resource) = null
+        def getMaximumResourceCapability = new Resource {
+          def getMemory = 512
+          def getVirtualCores = 2
+          def setMemory(memory: Int) {}
+          def setVirtualCores(vCores: Int) {}
+          def compareTo(o: Resource) = 0
+        }
+        def setMinimumResourceCapability(r: Resource) = null
+        def getMinimumResourceCapability = new Resource {
+          def getMemory = 128
+          def getVirtualCores = 1
+          def setMemory(memory: Int) {}
+          def setVirtualCores(vCores: Int) {}
+          def compareTo(o: Resource) = 0
+        }
+      }
+    }
+    def allocate(progressIndicator: Float): AllocateResponse = null
+    def unregisterApplicationMaster(appStatus: FinalApplicationStatus,
+      appMessage: String,
+      appTrackingUrl: String) {
+      this.status = appStatus
+    }
+    def addContainerRequest(req: ContainerRequest) {}
+    def removeContainerRequest(req: ContainerRequest) {}
+    def releaseAssignedContainer(containerId: ContainerId) {}
+    def getClusterAvailableResources(): Resource = null
+    def getClusterNodeCount() = 1
+
+    def init(config: Configuration) {}
+    def start() {}
+    def stop() {}
+    def register(listener: ServiceStateChangeListener) {}
+    def unregister(listener: ServiceStateChangeListener) {}
+    def getName(): String = ""
+    def getConfig() = null
+    def getServiceState() = null
+    def getStartTime() = 0L
+  }
+
+  @Test
+  def testLifecycleShouldRegisterOnInit {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "test", 1, 2)
+    state.rpcPort = 1
+    val saml = new SamzaAppMasterLifecycle(512, 2, state, amClient, new YarnConfiguration)
+    saml.onInit
+    assert(amClient.host == "test")
+    assert(amClient.port == 1)
+    assertFalse(saml.shouldShutdown)
+  }
+
+  @Test
+  def testLifecycleShouldUnregisterOnShutdown {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    state.status = FinalApplicationStatus.SUCCEEDED
+    new SamzaAppMasterLifecycle(128, 1, state, amClient, new YarnConfiguration).onShutdown
+    assert(amClient.status == FinalApplicationStatus.SUCCEEDED)
+  }
+
+  @Test
+  def testLifecycleShouldThrowAnExceptionOnReboot {
+    var gotException = false
+    try {
+      new SamzaAppMasterLifecycle(368, 1, null, amClient, new YarnConfiguration).onReboot
+    } catch {
+      // expected
+      case e: SamzaException => gotException = true
+    }
+    assert(gotException)
+  }
+
+  @Test
+  def testLifecycleShouldShutdownOnInvalidContainerSettings {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "test", 1, 2)
+    state.rpcPort = 1
+    List(new SamzaAppMasterLifecycle(768, 1, state, amClient, new YarnConfiguration),
+      new SamzaAppMasterLifecycle(0, 1, state, amClient, new YarnConfiguration),
+      new SamzaAppMasterLifecycle(368, 3, state, amClient, new YarnConfiguration),
+      new SamzaAppMasterLifecycle(768, 0, state, amClient, new YarnConfiguration)).map(saml => {
+        saml.onInit
+        assertTrue(saml.shouldShutdown)
+      })
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterService.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterService.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterService.scala
new file mode 100644
index 0000000..1099ca3
--- /dev/null
+++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterService.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.junit.Assert._
+import org.junit.Test
+import java.io.BufferedReader
+import java.net.URL
+import java.io.InputStreamReader
+import org.apache.hadoop.yarn.util.ConverterUtils
+
+class TestSamzaAppMasterService {
+  @Test
+  def testAppMasterDashboardShouldStart {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000002"), "", 1, 2)
+    val service = new SamzaAppMasterService(null, state, null, null)
+
+    // start the dashboard
+    service.onInit
+
+    // check to see if it's running
+    val url = new URL("http://127.0.0.1:%d/am" format state.rpcPort)
+    val is = url.openConnection().getInputStream();
+    val reader = new BufferedReader(new InputStreamReader(is));
+    var line: String = null;
+
+    do {
+      line = reader.readLine()
+    } while (line != null)
+
+    reader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterTaskManager.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterTaskManager.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterTaskManager.scala
new file mode 100644
index 0000000..6805052
--- /dev/null
+++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaAppMasterTaskManager.scala
@@ -0,0 +1,431 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.junit.Assert._
+import org.junit.Test
+import scala.collection.JavaConversions._
+import org.apache.samza.config.Config
+import org.apache.samza.config.MapConfig
+import org.apache.samza.Partition
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.util.ConverterUtils
+import scala.collection.JavaConversions._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse
+import org.apache.hadoop.yarn.client.AMRMClient
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.service._
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.yarn.api.records.NodeReport
+import TestSamzaAppMasterTaskManager._
+import org.apache.samza.system.SystemAdmin
+import org.apache.samza.system.SystemFactory
+import org.apache.samza.metrics.MetricsRegistry
+
+object TestSamzaAppMasterTaskManager {
+  def getContainer(containerId: ContainerId) = new Container {
+    def getId(): ContainerId = containerId
+    def setId(id: ContainerId) {}
+    def getNodeId(): NodeId = new NodeId {
+      var host = ""
+      var port = 12345
+      def getHost() = host
+      def setHost(host: String) = {
+        this.host = host
+      }
+      def getPort() = port
+      def setPort(port: Int) = {
+        this.port = port
+      }
+    }
+    def setNodeId(nodeId: NodeId) {}
+    def getNodeHttpAddress(): String = ""
+    def setNodeHttpAddress(nodeHttpAddress: String) {}
+    def getResource(): Resource = null
+    def setResource(resource: Resource) {}
+    def getPriority(): Priority = null
+    def setPriority(priority: Priority) {}
+    def getState(): ContainerState = null
+    def setState(state: ContainerState) {}
+    def getContainerToken(): ContainerToken = null
+    def setContainerToken(containerToken: ContainerToken) {}
+    def getContainerStatus(): ContainerStatus = null
+    def setContainerStatus(containerStatus: ContainerStatus) {}
+    def compareTo(c: Container): Int = containerId.compareTo(c.getId)
+  }
+
+  def getContainerStatus(containerId: ContainerId, exitCode: Int, diagnostic: String) = new ContainerStatus {
+    def getContainerId(): ContainerId = containerId
+    def setContainerId(containerId: ContainerId) {}
+    def getState(): ContainerState = null
+    def setState(state: ContainerState) {}
+    def getExitStatus(): Int = exitCode
+    def setExitStatus(exitStatus: Int) {}
+    def getDiagnostics() = diagnostic
+    def setDiagnostics(diagnostics: String) = {}
+  }
+
+  def getAmClient = (response: AllocateResponse) => new AMRMClient {
+    var requests: List[ContainerRequest] = List[ContainerRequest]()
+    var release: List[ContainerId] = List[ContainerId]()
+
+    def registerApplicationMaster(appHostName: String, appHostPort: Int, appTrackingUrl: String): RegisterApplicationMasterResponse = null
+    def allocate(progressIndicator: Float): AllocateResponse = response
+    def unregisterApplicationMaster(appStatus: FinalApplicationStatus, appMessage: String, appTrackingUrl: String) = null
+    def addContainerRequest(req: ContainerRequest) { requests ::= req }
+    def removeContainerRequest(req: ContainerRequest) {}
+    def releaseAssignedContainer(containerId: ContainerId) { release ::= containerId }
+    def getClusterAvailableResources(): Resource = null
+    def getClusterNodeCount() = 1
+
+    def init(config: Configuration) {}
+    def start() {}
+    def stop() {}
+    def register(listener: ServiceStateChangeListener) {}
+    def unregister(listener: ServiceStateChangeListener) {}
+    def getName(): String = ""
+    def getConfig() = null
+    def getServiceState() = null
+    def getStartTime() = 0L
+  }
+
+  def getAppMasterResponse(reboot: Boolean, containers: List[Container], completed: List[ContainerStatus]) =
+    new AllocateResponse {
+      def getAMResponse = new AMResponse {
+        def getReboot(): Boolean = reboot
+        def setReboot(reboot: Boolean) {}
+        def getResponseId() = 0
+        def setResponseId(responseId: Int) {}
+        def getAllocatedContainers() = containers
+        def setAllocatedContainers(containers: java.util.List[Container]) {}
+        def getAvailableResources(): Resource = null
+        def setAvailableResources(limit: Resource) {}
+        def getCompletedContainersStatuses() = completed
+        def setCompletedContainersStatuses(containers: java.util.List[ContainerStatus]) {}
+        def setUpdatedNodes(nodes: java.util.List[NodeReport]) {}
+        def getUpdatedNodes = null
+      }
+      def getNumClusterNodes = 1
+      def setNumClusterNodes(num: Int) {}
+      def setAMResponse(response: AMResponse) {}
+    }
+}
+
+class TestSamzaAppMasterTaskManager {
+  val config = new MapConfig(Map[String, String](
+    "yarn.container.count" -> "1",
+    "systems.test-system.samza.factory" -> "org.apache.samza.job.yarn.MockSystemFactory",
+    "yarn.container.memory.mb" -> "512",
+    "yarn.package.path" -> "/foo",
+    "task.inputs" -> "test-system.test-stream",
+    "systems.test-system.samza.key.serde" -> "org.apache.samza.serializers.JsonSerde",
+    "systems.test-system.samza.msg.serde" -> "org.apache.samza.serializers.JsonSerde",
+    "yarn.countainer.retry.count" -> "1",
+    "yarn.container.retry.window.ms" -> "1999999999"))
+
+  @Test
+  def testAppMasterShouldDefaultToOneContainerIfTaskCountIsNotSpecified {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    val taskManager = new SamzaAppMasterTaskManager(clock, config, state, null, new YarnConfiguration)
+    assert(state.taskCount == 1)
+  }
+
+  @Test
+  def testAppMasterShouldStopWhenContainersFinish {
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    val taskManager = new SamzaAppMasterTaskManager(clock, config, state, null, new YarnConfiguration)
+
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onContainerCompleted(getContainerStatus(state.containerId, 0, ""))
+    assert(taskManager.shouldShutdown == true)
+    assert(state.completedTasks == 1)
+    assert(state.taskCount == 1)
+    assert(state.status.equals(FinalApplicationStatus.SUCCEEDED))
+  }
+
+  @Test
+  def testAppMasterShouldRequestANewContainerWhenATaskFails {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    val taskManager = new SamzaAppMasterTaskManager(clock, config, state, amClient, new YarnConfiguration) {
+      override def startContainer(packagePath: Path, container: Container, ugi: UserGroupInformation, env: Map[String, String], cmds: String*) {
+        // Do nothing.
+      }
+    }
+
+    assert(taskManager.shouldShutdown == false)
+    val container2 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000002")
+    taskManager.onInit
+    taskManager.onContainerAllocated(getContainer(container2))
+    taskManager.onContainerCompleted(getContainerStatus(container2, 1, "expecting a failure here"))
+    assert(taskManager.shouldShutdown == false)
+    // 2. First is from onInit, second is from onContainerCompleted, since it failed.
+    assertEquals(2, amClient.requests.size)
+    assertEquals(0, amClient.release.size)
+    assertFalse(taskManager.shouldShutdown)
+    // Now trigger an AM shutdown since our retry count is 1, and we're failing twice
+    taskManager.onContainerAllocated(getContainer(container2))
+    taskManager.onContainerCompleted(getContainerStatus(container2, 1, "expecting a failure here"))
+    assertEquals(2, amClient.requests.size)
+    assertEquals(0, amClient.release.size)
+    assertTrue(taskManager.shouldShutdown)
+  }
+
+  @Test
+  def testAppMasterShouldRequestANewContainerWhenATaskIsReleased {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    state.taskCount = 2
+    var containersRequested = 0
+    var containersStarted = 0
+    val taskManager = new SamzaAppMasterTaskManager(clock, config, state, amClient, new YarnConfiguration) {
+      override def startContainer(packagePath: Path, container: Container, ugi: UserGroupInformation, env: Map[String, String], cmds: String*) {
+        containersStarted += 1
+      }
+
+      override def requestContainers(memMb: Int, cpuCores: Int, containers: Int) {
+        containersRequested += 1
+        super.requestContainers(memMb, cpuCores, containers)
+      }
+    }
+    val container2 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000002")
+    val container3 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000003")
+
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onInit
+    assert(taskManager.shouldShutdown == false)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 0)
+
+    // allocate container 2
+    taskManager.onContainerAllocated(getContainer(container2))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(containersRequested == 1)
+    assert(containersStarted == 1)
+
+    // allocate an extra container, which the AM doesn't need, and should be released
+    taskManager.onContainerAllocated(getContainer(container3))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 1)
+    assert(amClient.release(0).equals(container3))
+
+    // reset the helper state, so we can make sure that releasing the container (next step) doesn't request more resources
+    amClient.requests = List()
+    amClient.release = List()
+
+    // now release the container, and make sure the AM doesn't ask for more
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onContainerCompleted(getContainerStatus(container3, -100, "pretend the container was released"))
+    assert(taskManager.shouldShutdown == false)
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(amClient.requests.size == 0)
+    assert(amClient.release.size == 0)
+
+    // pretend container 2 is released due to an NM failure, and make sure that the AM requests a new container
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onContainerCompleted(getContainerStatus(container2, -100, "pretend the container was 'lost' due to an NM failure"))
+    assert(taskManager.shouldShutdown == false)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 0)
+  }
+
+  @Test
+  def testAppMasterShouldWorkWithMoreThanOneContainer {
+    val map = new java.util.HashMap[String, String](config)
+    map.put("yarn.container.count", "2")
+    val newConfig = new MapConfig(map)
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    state.taskCount = 2
+    var containersStarted = 0
+    val taskManager = new SamzaAppMasterTaskManager(clock, newConfig, state, amClient, new YarnConfiguration) {
+      override def startContainer(packagePath: Path, container: Container, ugi: UserGroupInformation, env: Map[String, String], cmds: String*) {
+        containersStarted += 1
+      }
+    }
+    val container2 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000002")
+    val container3 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000003")
+
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onInit
+    assert(taskManager.shouldShutdown == false)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 0)
+    taskManager.onContainerAllocated(getContainer(container2))
+    assert(state.neededContainers == 1)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 1)
+    assert(containersStarted == 1)
+    taskManager.onContainerAllocated(getContainer(container3))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 2)
+    assert(state.taskPartitions.size == 2)
+    assert(state.unclaimedTasks.size == 0)
+    assert(containersStarted == 2)
+
+    // container2 finishes successfully
+    taskManager.onContainerCompleted(getContainerStatus(container2, 0, ""))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(state.completedTasks == 1)
+
+    // container3 fails
+    taskManager.onContainerCompleted(getContainerStatus(container3, 1, "expected failure here"))
+    assert(state.neededContainers == 1)
+    assert(state.runningTasks.size == 0)
+    assert(state.taskPartitions.size == 0)
+    assert(state.unclaimedTasks.size == 1)
+    assert(state.completedTasks == 1)
+    assert(taskManager.shouldShutdown == false)
+
+    // container3 is re-allocated
+    taskManager.onContainerAllocated(getContainer(container3))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(containersStarted == 3)
+
+    // container3 finishes sucecssfully
+    taskManager.onContainerCompleted(getContainerStatus(container3, 0, ""))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 0)
+    assert(state.taskPartitions.size == 0)
+    assert(state.unclaimedTasks.size == 0)
+    assert(state.completedTasks == 2)
+    assert(taskManager.shouldShutdown == true)
+  }
+
+  @Test
+  def testAppMasterShouldReleaseExtraContainers {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val state = new SamzaAppMasterState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "", 1, 2)
+    var containersRequested = 0
+    var containersStarted = 0
+    val taskManager = new SamzaAppMasterTaskManager(clock, config, state, amClient, new YarnConfiguration) {
+      override def startContainer(packagePath: Path, container: Container, ugi: UserGroupInformation, env: Map[String, String], cmds: String*) {
+        containersStarted += 1
+      }
+
+      override def requestContainers(memMb: Int, cpuCores: Int, containers: Int) {
+        containersRequested += 1
+        super.requestContainers(memMb, cpuCores, containers)
+      }
+    }
+    val container2 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000002")
+    val container3 = ConverterUtils.toContainerId("container_1350670447861_0003_01_000003")
+
+    assert(taskManager.shouldShutdown == false)
+    taskManager.onInit
+    assert(taskManager.shouldShutdown == false)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 0)
+    assert(state.neededContainers == 1)
+    assert(state.runningTasks.size == 0)
+    assert(state.taskPartitions.size == 0)
+    assert(state.unclaimedTasks.size == 1)
+    taskManager.onContainerAllocated(getContainer(container2))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(containersRequested == 1)
+    assert(containersStarted == 1)
+    taskManager.onContainerAllocated(getContainer(container3))
+    assert(state.neededContainers == 0)
+    assert(state.runningTasks.size == 1)
+    assert(state.taskPartitions.size == 1)
+    assert(state.unclaimedTasks.size == 0)
+    assert(containersRequested == 1)
+    assert(containersStarted == 1)
+    assert(amClient.requests.size == 1)
+    assert(amClient.release.size == 1)
+    assert(amClient.release(0).equals(container3))
+  }
+
+  @Test
+  def testPartitionsShouldWorkWithMoreTasksThanPartitions {
+    val onePartition = Set(new Partition(0))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(0, 2, onePartition).equals(Set(new Partition(0))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(1, 2, onePartition).equals(Set()))
+  }
+
+  @Test
+  def testPartitionsShouldWorkWithMorePartitionsThanTasks {
+    val fivePartitions = (0 until 5).map(new Partition(_)).toSet
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(0, 2, fivePartitions).equals(Set(new Partition(0), new Partition(2), new Partition(4))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(1, 2, fivePartitions).equals(Set(new Partition(1), new Partition(3))))
+  }
+
+  @Test
+  def testPartitionsShouldWorkWithTwelvePartitionsAndFiveContainers {
+    val fivePartitions = (0 until 12).map(new Partition(_)).toSet
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(0, 5, fivePartitions).equals(Set(new Partition(0), new Partition(5), new Partition(10))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(1, 5, fivePartitions).equals(Set(new Partition(1), new Partition(6), new Partition(11))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(2, 5, fivePartitions).equals(Set(new Partition(2), new Partition(7))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(3, 5, fivePartitions).equals(Set(new Partition(3), new Partition(8))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(4, 5, fivePartitions).equals(Set(new Partition(4), new Partition(9))))
+  }
+
+  @Test
+  def testPartitionsShouldWorkWithEqualPartitionsAndTasks {
+    val twoPartitions = (0 until 2).map(new Partition(_)).toSet
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(0, 2, twoPartitions).equals(Set(new Partition(0))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(1, 2, twoPartitions).equals(Set(new Partition(1))))
+    assert(SamzaAppMasterTaskManager.getPartitionsForTask(0, 1, Set(new Partition(0))).equals(Set(new Partition(0))))
+  }
+
+  val clock = () => System.currentTimeMillis
+}
+
+class MockSystemFactory extends SystemFactory {
+  def getConsumer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    throw new RuntimeException("Hmm. Not implemented.")
+  }
+
+  def getProducer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    throw new RuntimeException("Hmm. Not implemented.")
+  }
+
+  def getAdmin(systemName: String, config: Config) = {
+    new MockSinglePartitionManager
+  }
+
+}
+
+class MockSinglePartitionManager extends SystemAdmin {
+  def getPartitions(streamName: String) = Set(new Partition(0))
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestYarnAppMaster.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestYarnAppMaster.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestYarnAppMaster.scala
new file mode 100644
index 0000000..0040648
--- /dev/null
+++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestYarnAppMaster.scala
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.junit.Assert._
+import org.junit.Test
+import TestSamzaAppMasterTaskManager._
+import org.apache.hadoop.yarn.api.records.Container
+import org.apache.hadoop.yarn.api.records.ContainerStatus
+import org.apache.hadoop.yarn.api.records.ResourceRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.api.records.AMResponse
+import org.apache.hadoop.yarn.api.records.ContainerId
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse
+
+class TestYarnAppMaster {
+  @Test
+  def testAppMasterShouldShutdown {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val listener = new YarnAppMasterListener {
+      var init = 0
+      var shutdown = 0
+      var allocated = 0
+      var complete = 0
+      override def shouldShutdown = true
+      override def onInit() {
+        init += 1
+      }
+      override def onShutdown() {
+        shutdown += 1
+      }
+      override def onContainerAllocated(container: Container) {
+        allocated += 1
+      }
+      override def onContainerCompleted(containerStatus: ContainerStatus) {
+        complete += 1
+      }
+    }
+    new YarnAppMaster(List(listener), amClient).run
+    assert(listener.init == 1)
+    assert(listener.shutdown == 1)
+  }
+
+  @Test
+  def testAppMasterShouldShutdownWithFailingListener {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val listener1 = new YarnAppMasterListener {
+      var shutdown = 0
+      override def shouldShutdown = true
+      override def onShutdown() {
+        shutdown += 1
+        throw new RuntimeException("Some weird failure")
+      }
+    }
+    val listener2 = new YarnAppMasterListener {
+      var shutdown = 0
+      override def shouldShutdown = true
+      override def onShutdown() {
+        shutdown += 1
+      }
+    }
+    // listener1 will throw an exception in shutdown, and listener2 should still get called 
+    new YarnAppMaster(List(listener1, listener2), amClient).run
+    assert(listener1.shutdown == 1)
+    assert(listener2.shutdown == 1)
+  }
+
+  @Test
+  def testAppMasterShouldShutdownWithInterrupt {
+    val amClient = getAmClient(getAppMasterResponse(false, List(), List()))
+    val listener = new YarnAppMasterListener {
+      var init = 0
+      var shutdown = 0
+      override def shouldShutdown = false
+      override def onInit() {
+        init += 1
+      }
+      override def onShutdown() {
+        shutdown += 1
+      }
+    }
+    val am = new YarnAppMaster(List(listener), amClient)
+    val thread = new Thread {
+      override def run {
+        am.run
+      }
+    }
+    thread.start
+    thread.interrupt
+    thread.join
+    assert(listener.init == 1)
+    assert(listener.shutdown == 1)
+  }
+
+  @Test
+  def testAppMasterShouldForwardAllocatedAndCompleteContainers {
+    val amClient = getAmClient(getAppMasterResponse(false, List(getContainer(null)), List(getContainerStatus(null, 1, null))))
+    val listener = new YarnAppMasterListener {
+      var allocated = 0
+      var complete = 0
+      override def shouldShutdown = (allocated == 1 && complete == 1)
+      override def onContainerAllocated(container: Container) {
+        allocated += 1
+      }
+      override def onContainerCompleted(containerStatus: ContainerStatus) {
+        complete += 1
+      }
+    }
+    new YarnAppMaster(List(listener), amClient).run
+    assert(listener.allocated == 1)
+    assert(listener.complete == 1)
+  }
+
+  @Test
+  def testAppMasterShouldReboot {
+    val amClient = getAmClient(getAppMasterResponse(true, List(), List()))
+    val listener = new YarnAppMasterListener {
+      var reboot = 0
+      override def shouldShutdown = reboot == 1
+      override def onReboot() {
+        reboot += 1
+      }
+    }
+    new YarnAppMaster(List(listener), amClient).run
+    assert(listener.reboot == 1)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
new file mode 100644
index 0000000..9e7a4ea
--- /dev/null
+++ b/settings.gradle
@@ -0,0 +1,7 @@
+include 'samza-api', 'samza-core', 'samza-kafka', 'samza-kv', 'samza-serializers', 'samza-shell', 'samza-yarn', 'samza-test'
+
+rootProject.children.each {
+  if (it.name != 'samza-api' && it.name != 'samza-shell') {
+    it.name = it.name + "_" + scalaVersion
+  }
+}


[14/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/coding-guide.md
----------------------------------------------------------------------
diff --git a/docs/contribute/coding-guide.md b/docs/contribute/coding-guide.md
new file mode 100644
index 0000000..cdcb0ea
--- /dev/null
+++ b/docs/contribute/coding-guide.md
@@ -0,0 +1,97 @@
+---
+layout: page
+title: Coding Guide
+---
+
+<!-- TODO link to hudson when we have an apache hudson boxes. -->
+
+These guidelines are meant to encourage consistency and best practices amongst people working on the Samza code base. They should be observed unless there is a compelling reason to ignore them.
+
+### Basic Stuff
+
+* Avoid cryptic abbreviations. Single letter variable names are fine in very short methods with few variables, otherwise make them informative.
+* Clear code is preferable to comments. When possible make your naming so good you don't need comments. When that isn't possible comments should be thought of as mandatory, write them to be read.
+* Logging, configuration, and public APIs are our "UI". Make them pretty, consistent, and usable.
+* There is not a maximum line length (certainly not 80 characters, we don't work on punch cards any more), but be reasonable.
+* Don't be sloppy. Don't check in commented out code: we use version control, it is still there in the history. Don't leave TODOs in the code or FIXMEs if you can help it. Don't leave println statements in the code. Hopefully this is all obvious.
+* We want people to use our stuff, which means we need clear, correct documentation. User documentation should be considered a part of any user-facing the feature, just like unit tests or performance results.
+* Don't duplicate code (duh).
+* Any API that's user-facing (something that a Samza job could use) should be defined in samza-api as a Java interface. Scala is for implementation only.
+
+### Scala
+
+We are following the style guide given here (though not perfectly). Below are some specifics worth noting:
+
+* Scala is a very flexible language. Use restraint. Magic cryptic one-liners do not impress us, readability impresses us.
+* Use vals when possible.
+* Use private when possible for member variables.
+* Method and member variable names should be in camel case with an initial lower case character like aMethodName.
+* Constants should be camel case with an initial capital LikeThis not LIKE_THIS.
+* Prefer a single top-level class per file for ease of finding things.
+* Do not use semi-colons unless required.
+* Avoid getters and setters - stick to plain vals or vars instead. If (later on) you require a custom setter (or getter) for a var named myVar then add a shadow var myVar\_underlying and override the setter (def myVar =) and the getter (def myVar = myVar\_underlying).
+* Perfer Option to null in scala APIs.
+* Use named arguments when passing in literal values if the meaning is at all unclear, for example instead of Utils.delete(true) prefer Utils.delete(recursive=true).
+* Indentation is 2 spaces and never tabs. One could argue the right amount of indentation, but 2 seems to be standard for Scala and consistency is best here since there is clearly no "right" way.
+* Include the optional parenthesis on a no-arg method only if the method has a side-effect, otherwise omit them. For example fileChannel.force() and fileChannel.size. This helps emphasize that you are calling the method for the side effect, which is changing some state, not just getting the return value.
+* Prefer case classes to tuples in important APIs to make it clear what the intended contents are.
+
+### Logging
+
+* We use [grizzled-slf4j](http://software.clapper.org/grizzled-slf4j/) for Scala logging, and [SLF4J](http://www.slf4j.org/) (with [Log4J](http://logging.apache.org/log4j/2.x/)) for Java.
+* Logging is one third of our "UI" and it should be taken seriously. Please take the time to assess the logs when making a change to ensure that the important things are getting logged and there is no junk there.
+* Don't include a stack trace in INFO, or above, unless there is really something wrong. Stack traces in logs should signal something is wrong, not be informative. If you want to be informative, write an actual log line that say's what's important, and save the stack trace for DEBUG.
+* Logging statements should be complete sentences with proper capitalization that are written to be read by a person not necessarily familiar with the source code. * It is fine to put in hacky little logging statements when debugging, but either clean them up or remove them before checking in. So logging something like "INFO: entering SyncProducer send()" is not appropriate.
+* Logging should not mention class names or internal variables.
+* There are six levels of logging TRACE, DEBUG, INFO, WARN, ERROR, and FATAL, they should be used as follows.
+  * INFO is the level you should assume the software will be run in. INFO messages are things which are not bad but which the user will definitely want to know about every time they occur.
+  * TRACE and DEBUG are both things you turn on when something is wrong and you want to figure out what is going on. DEBUG should not be so fine grained that it will seriously effect the performance of the Samza job. TRACE can be anything.
+  * WARN and ERROR indicate something that is bad. Use WARN if you aren't totally sure it is bad, and ERROR if you are.
+  * Use FATAL only right before calling System.exit().
+
+### Metrics
+
+* Metrics should be taken seriously. The goal with metrics is to provide enough information that users can determine that their Samza job is running properly.
+* We have a metrics package in samza-api. It supports counters and gauges, and should be used for all features.
+* Any new features should come with appropriate metrics to know the feature is working correctly. This is at least as important as unit tests as it verifies production.
+* Metric naming should be of the form: group=samza.core.task.runner, name=UnprocessedMessages.
+
+### Unit Tests
+
+* New patches should come with unit tests that verify the functionality being added.
+* Unit tests are first rate code, and should be treated like it. They should not contain code duplication, cryptic hackery, or anything like that.
+* Unit tests should test the least amount of code possible, don't start Kafka or YARN unless there is no other way to test a single class or small group of classes in isolation.
+* Tests should not depend on any external resources, they need to set up and tear down their own stuff. This means if you want zookeeper it needs to be started and stopped, you can't depend on it already being there. Likewise if you need a file with some data in it, you need to write it in the beginning of the test and delete it (pass or fail).
+* Do not use sleep or other timing assumptions in tests, it is always, always, always wrong and will fail intermittently on any test server with other things going on that causes delays. Write tests in such a way that they are not timing dependent. Seriously. One thing that will help this is to never directly use the system clock in code (i.e. System.currentTimeMillis) but instead to use getTime: () => Long, so that time can be mocked.
+* It must be possible to run the tests in parallel, without having them collide. This is a practical thing to allow multiple branches to CI on a single CI server. This means you can't hard code directories or ports or things like that in tests because two instances will step on each other.
+
+### Configuration
+
+* Configuration is the final third of our "UI".
+* All configuration names that define time must end with .ms (e.g. foo.bar.ms=1000).
+* All configuration names that define a byte size must end with .bytes (e.g. foo.bar.bytes=1000).
+* All configuration names that define a URI must end with .uri (e.g. yarn.package.uri).
+* All configuration names that support a CSV list must end with .list (e.g. task.input.stream.list)
+* All configuration names that define a class must end with .class (e.g. task.command.class).
+* All configuration names that define a factory class must end with .factory.class (e.g. systems.kafka.consumer.factory.class).
+* Configuration will always be defined as simple key/value pairs (e.g. a=b).
+* When configuration is related, it must be grouped using the same prefix (e.g. yarn.container.count=1, yarn.container.memory.bytes=1073741824).
+* When configuration must be defined multiple times, the key should be parameterized (e.g. systems.kafka.consumer.factory=x, systems.kestrel.consumer.factory=y). *When such configuration must be referred to, its parameter should be used (e.g. foo.bar.system=kafka, foo.bar.system=kestrel).
+* All getter methods must be a camel case match with their configuration names (e.g. yarn.package.uri and getYarnPackageUri).
+* Reading configuration should only be done in factories and main methods. Don't pass Config objects around.
+* Names should be thought through from the point of view of the person using the config, but often programmers choose configuration names that make sense for someone reading the code.
+* Often the value that makes most sense in configuration is not the one most useful to program with. For example, let's say you want to throttle I/O to avoid using up all the I/O bandwidth. The easiest thing to implement is to give a "sleep time" configuration that let's the program sleep after doing I/O to throttle down its rate. But notice how hard it is to correctly use this configuration parameter, the user has to figure out the rate of I/O on the machine, and then do a bunch of arithmetic to calculate the right sleep time to give the desired rate of I/O on the system. It is much, much, much better to just have the user configure the maximum I/O rate they want to allow (say 5MB/sec) and then calculate the appropriate sleep time from that and the actual I/O rate. Another way to say this is that configuration should always be in terms of the quantity that the user knows, not the quantity you want to use.
+* Configuration is the answer to problems we can't solve up front for some reason--if there is a way to just choose a best value do that instead.
+* Configuration should come from the job-level properties file. No additional sources of config (environment variables, system properties, etc) should be added as these usually inhibit running multiple instances of a broker on one machine.
+
+### Concurrency
+
+* Encapsulate synchronization. That is, locks should be private member variables within a class and only one class or method should need to be examined to verify the correctness of the synchronization strategy.
+* There are a number of gotchas with threads and threadpools: is the daemon flag set appropriately for your threads? are your threads being named in a way that will distinguish their purpose in a thread dump? What happens when the number of queued tasks hits the limit (do you drop stuff? do you block?).
+* Prefer the java.util.concurrent packages to either low-level wait-notify, custom locking/synchronization, or higher level scala-specific primitives. The util.concurrent stuff is well thought out and actually works correctly. There is a generally feeling that threads and locking are not going to be the concurrency primitives of the future because of a variety of well-known weaknesses they have. This is probably true, but they have the advantage of actually being mature enough to use for high-performance software right now; their well-known deficiencies are easily worked around by equally well known best-practices. So avoid actors, software transactional memory, tuple spaces, or anything else not written by Doug Lea and used by at least a million other productions systems. :-)
+
+### Backwards Compatibility
+
+* Samza uses [Semantic Versioning](http://semver.org/).
+* Backwards incompatible API changes, config changes, or library upgrades should only happen between major revision changes, or when the major revision is 0.
+* The samza-api and samza-core packages should never depend on anything except logging and JSON (jackson) libraries. Prefer granular packages that isolate dependencies, rather than larger packages that group un-related dependencies together.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/disclaimer.md
----------------------------------------------------------------------
diff --git a/docs/contribute/disclaimer.md b/docs/contribute/disclaimer.md
new file mode 100644
index 0000000..3fe446f
--- /dev/null
+++ b/docs/contribute/disclaimer.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Disclaimer
+---
+
+Apache Samza is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/projects.md
----------------------------------------------------------------------
diff --git a/docs/contribute/projects.md b/docs/contribute/projects.md
new file mode 100644
index 0000000..364c388
--- /dev/null
+++ b/docs/contribute/projects.md
@@ -0,0 +1,20 @@
+---
+layout: page
+title: Projects
+---
+
+<!-- TODO link to jira newbie/meaty when we have them. -->
+
+### Newbies
+
+We tag bugs in [JIRA](https://issues.apache.org/jira/browse/SAMZA) with "newbie" that are good for people getting started with the code base. You can find the list here.
+
+### Larger Chunks of Work
+
+More meaty projects are here. The process for working on a large project is:
+
+1. Instigate discussion on the [JIRA](https://issues.apache.org/jira/browse/SAMZA).
+2. Write a [SEP](seps.html) (Samza Enhancement Proposal).
+3. Request feedback for the [SEP](seps.html) on the Jira and the samza-dev mailing list.
+4. Come to an agreement on design.
+5. Implement design.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/rules.md
----------------------------------------------------------------------
diff --git a/docs/contribute/rules.md b/docs/contribute/rules.md
new file mode 100644
index 0000000..3bc42c3
--- /dev/null
+++ b/docs/contribute/rules.md
@@ -0,0 +1,22 @@
+---
+layout: page
+title: Rules
+---
+
+<!-- TODO link to jira when we have an apache jira. -->
+
+We are always very happy to have code contributions whether for trivial cleanups or big new features. In general, patches should include:
+
+* Code change
+* Unit tests
+* Javadocs
+* Metrics
+* Logging
+
+To submit a patch for inclusion please do the following:
+
+* If you are working on a big new feature ([project](projects.html)), follow the steps outlined on the [SEPs](/contribute/seps.html) page.
+* Create a patch that applies cleanly against trunk.
+* Make sure you have observed the recommendations in the style guide.
+* Open a JIRA ticket describing the patch and attach your patch to the JIRA.
+* Nag us if we don't follow up on your JIRA in a timely fashion.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/seps.md
----------------------------------------------------------------------
diff --git a/docs/contribute/seps.md b/docs/contribute/seps.md
new file mode 100644
index 0000000..71342fd
--- /dev/null
+++ b/docs/contribute/seps.md
@@ -0,0 +1,27 @@
+---
+layout: page
+title: Samza Enhancement Proposals
+---
+
+<!-- TODO link to jira when we have an apache jira. -->
+
+When making larger changes to Samza, or working on a [project](/contribute/projects.html), please write a Samza Enhancement Proposal (SEP) on the Samza wiki. The goal of the SEP is to:
+
+1. Define the problem you're trying to solve
+2. Propose a solution, and explore alternatives.
+3. Instigate discussion on the issue.
+4. Archive design documents for future use.
+
+### How to Write a SEP
+
+SEPs are stored on the Samza wiki. To write a SEP, create a new sub page on the wiki with a title formatted as SEP-# (e.g. SEP-24).
+
+There is no set single format for a SEP, but it's common to include:
+
+1. Table of Contents
+2. Introduction
+3. Definition of problem
+4. Possible solutions
+5. Opinion on best solution
+6. Proposed metrics to add.
+7. A link to the [JIRA](https://issues.apache.org/jira/browse/SAMZA).

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/css/main.css
----------------------------------------------------------------------
diff --git a/docs/css/main.css b/docs/css/main.css
new file mode 100755
index 0000000..ae62572
--- /dev/null
+++ b/docs/css/main.css
@@ -0,0 +1,161 @@
+html {
+	height: 100%;
+}
+
+body {
+	margin: 0px;
+	font-family: helvetica;
+	height: 100%;
+}
+
+.header {
+	background-color: #F5F4F0;
+}
+
+.page {
+	margin-left: 15%;
+	width: 70%;
+}
+
+.body {
+	line-height: 150%;
+	margin-left: 16.67em;
+	font-size: .9em;
+}
+
+div.logo {
+	display: inline-block;
+	width: 12em;
+	font-family: 'Ropa Sans', sans-serif;
+	background-color: #FF0000;
+	padding: .5em 1em;
+}
+
+div.nav-footer {
+	margin: .5em 0em;
+}
+
+.right {
+	float: right;
+}
+
+.left {
+	float: left;
+}
+
+.top-icons {
+	float: right;
+	padding-top: .5em;
+}
+
+.top-icon {
+	font-size: 3em;
+	color: #d3d2d0;
+	margin-left: .2em;
+}
+
+.committer-icon {
+  font-size: 1.1em;
+	margin-right: .3em;
+}
+
+a.logo {
+	font-size: 3em;
+	color: white;
+	text-decoration: none;
+}
+
+.menu {
+	float: left;
+}
+
+.menu-inner {
+	width: 12em;
+	padding: 0em 1em;
+	line-height: 120%;
+}
+
+.menu-title {
+	font-weight: bold;
+}
+
+.menu-category {
+	padding: .5em 0em 1.5em 0em;
+	font-size: .8em;
+}
+
+h2 {
+	margin: 1em 0em;
+}
+
+h4 {
+	margin: .5em 0em;
+}
+
+a {
+	text-decoration: none;
+	color: #1B91E0;
+}
+
+small {
+	color: #ccc;
+}
+
+img[alt=diagram] {
+	width: 90%;
+}
+
+img[alt=diagram-medium] {
+	width: 40%;
+}
+
+img[alt=diagram-small] {
+	width: 25%;
+}
+
+code {
+	display: block;
+	line-height: 120%;
+	overflow-x: scroll;
+	width: 100%;
+	padding: 1em;
+	background-color: #eee;
+}
+
+table {
+	width: 100%;
+	text-align: left;
+}
+
+div.documentation-second-level {
+	margin-left: 20px;
+}
+
+/* footer */
+html, body, .container {
+  height: 100%;
+}
+
+body > .container {
+  height: auto;
+  min-height: 100%;
+}
+
+.footer {
+  clear: both;
+  position: relative;
+  z-index: 10;
+  height: 5em;
+  margin-top: -5em;
+  background-color: #F5F4F0;
+}
+
+.footer-content {
+  padding: 1em;
+}
+
+.container-inner {
+  padding-bottom: 5em;
+}
+
+/* end footer */

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/comparisons/mupd8-samza.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/comparisons/mupd8-samza.png b/docs/img/0.7.0/learn/documentation/comparisons/mupd8-samza.png
new file mode 100644
index 0000000..5882cea
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/comparisons/mupd8-samza.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/comparisons/mupd8.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/comparisons/mupd8.png b/docs/img/0.7.0/learn/documentation/comparisons/mupd8.png
new file mode 100644
index 0000000..ca79982
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/comparisons/mupd8.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png b/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png
new file mode 100644
index 0000000..cfd85e1
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/container/checkpointing.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/checkpointing.png b/docs/img/0.7.0/learn/documentation/container/checkpointing.png
new file mode 100644
index 0000000..aa36240
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/container/checkpointing.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/container/job-flow.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/job-flow.png b/docs/img/0.7.0/learn/documentation/container/job-flow.png
new file mode 100644
index 0000000..046f9e3
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/container/job-flow.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/container/metrics.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/metrics.png b/docs/img/0.7.0/learn/documentation/container/metrics.png
new file mode 100644
index 0000000..8be2a38
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/container/metrics.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png b/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png
new file mode 100644
index 0000000..68b753f
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/dag.graffle
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/dag.graffle b/docs/img/0.7.0/learn/documentation/introduction/dag.graffle
new file mode 100644
index 0000000..d743ea4
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/introduction/dag.graffle
@@ -0,0 +1,1009 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN" "http://www.apple.com/DTDs/PropertyList-1.0.dtd">
+<plist version="1.0">
+<dict>
+	<key>ActiveLayerIndex</key>
+	<integer>0</integer>
+	<key>ApplicationVersion</key>
+	<array>
+		<string>com.omnigroup.OmniGrafflePro.MacAppStore</string>
+		<string>139.18</string>
+	</array>
+	<key>AutoAdjust</key>
+	<true/>
+	<key>BackgroundGraphic</key>
+	<dict>
+		<key>Bounds</key>
+		<string>{{0, 0}, {576.00002479553223, 733}}</string>
+		<key>Class</key>
+		<string>SolidGraphic</string>
+		<key>ID</key>
+		<integer>2</integer>
+		<key>Style</key>
+		<dict>
+			<key>shadow</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+			<key>stroke</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+		</dict>
+	</dict>
+	<key>BaseZoom</key>
+	<integer>0</integer>
+	<key>CanvasOrigin</key>
+	<string>{0, 0}</string>
+	<key>ColumnAlign</key>
+	<integer>1</integer>
+	<key>ColumnSpacing</key>
+	<real>36</real>
+	<key>CreationDate</key>
+	<string>2013-07-28 22:58:14 +0000</string>
+	<key>Creator</key>
+	<string>Jay Kreps</string>
+	<key>DisplayScale</key>
+	<string>1 0/72 in = 1 0/72 in</string>
+	<key>GraphDocumentVersion</key>
+	<integer>8</integer>
+	<key>GraphicsList</key>
+	<array>
+		<dict>
+			<key>Bounds</key>
+			<string>{{43.000001907348633, 12}, {208, 22}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>39</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs36 \cf0 A Multjob Dataflow Graph}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>37</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>38</integer>
+			<key>Points</key>
+			<array>
+				<string>{236.00000190734863, 144}</string>
+				<string>{231.00000190734863, 224}</string>
+				<string>{147.00000190734863, 292}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>23</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{113.00000190734863, 292}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>37</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream F}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>37</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>36</integer>
+			<key>Points</key>
+			<array>
+				<string>{147.00000190734863, 268}</string>
+				<string>{147.00000190734863, 292}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>6</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>6</integer>
+			</dict>
+			<key>ID</key>
+			<integer>35</integer>
+			<key>Points</key>
+			<array>
+				<string>{147.00000190734863, 204}</string>
+				<string>{147.00000190734863, 232}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>27</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>6</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>34</integer>
+			<key>Points</key>
+			<array>
+				<string>{60.000001907348633, 204}</string>
+				<string>{147.00000190734863, 232}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>31</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>31</integer>
+			</dict>
+			<key>ID</key>
+			<integer>33</integer>
+			<key>Points</key>
+			<array>
+				<string>{60.000001907348633, 144}</string>
+				<string>{60.000001907348633, 177}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>28</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{26.000001907348633, 177}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>31</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream D}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{113.00000190734863, 177}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>27</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream E}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>27</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>29</integer>
+			<key>Points</key>
+			<array>
+				<string>{236.00000190734863, 144}</string>
+				<string>{147.00000190734863, 177}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>23</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>27</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>28</integer>
+			<key>Points</key>
+			<array>
+				<string>{60.000001907348633, 144}</string>
+				<string>{147.00000190734863, 177}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>23</integer>
+			</dict>
+			<key>ID</key>
+			<integer>26</integer>
+			<key>Points</key>
+			<array>
+				<string>{236.00000190734863, 72}</string>
+				<string>{236.00000190734863, 108}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>22</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>23</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>25</integer>
+			<key>Points</key>
+			<array>
+				<string>{151.00000190734863, 72}</string>
+				<string>{236.00000190734863, 108}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>21</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>5</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>24</integer>
+			<key>Points</key>
+			<array>
+				<string>{151.00000190734863, 72}</string>
+				<string>{60.000003814697266, 108}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>21</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{191.00000190734863, 108}, {90, 36}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>23</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Diamond</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Job 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{202.00000190734863, 45}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>22</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream C}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{117.00000190734863, 45}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>21</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream B}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>5</integer>
+			</dict>
+			<key>ID</key>
+			<integer>20</integer>
+			<key>Points</key>
+			<array>
+				<string>{60.000001907348633, 72}</string>
+				<string>{60.000003814697266, 108}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>19</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{26.000001907348633, 45}, {68, 27}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>19</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Stream A}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{102.00000190734863, 232}, {90, 36}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>6</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Diamond</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Job B}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{15.000003814697266, 108}, {90, 36}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>5</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Diamond</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs22 \cf0 Job 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+	</array>
+	<key>GridInfo</key>
+	<dict/>
+	<key>GuidesLocked</key>
+	<string>NO</string>
+	<key>GuidesVisible</key>
+	<string>YES</string>
+	<key>HPages</key>
+	<integer>1</integer>
+	<key>ImageCounter</key>
+	<integer>1</integer>
+	<key>KeepToScale</key>
+	<false/>
+	<key>Layers</key>
+	<array>
+		<dict>
+			<key>Lock</key>
+			<string>NO</string>
+			<key>Name</key>
+			<string>Layer 1</string>
+			<key>Print</key>
+			<string>YES</string>
+			<key>View</key>
+			<string>YES</string>
+		</dict>
+	</array>
+	<key>LayoutInfo</key>
+	<dict>
+		<key>Animate</key>
+		<string>NO</string>
+		<key>circoMinDist</key>
+		<real>18</real>
+		<key>circoSeparation</key>
+		<real>0.0</real>
+		<key>layoutEngine</key>
+		<string>dot</string>
+		<key>neatoSeparation</key>
+		<real>0.0</real>
+		<key>twopiSeparation</key>
+		<real>0.0</real>
+	</dict>
+	<key>LinksVisible</key>
+	<string>NO</string>
+	<key>MagnetsVisible</key>
+	<string>NO</string>
+	<key>MasterSheets</key>
+	<array/>
+	<key>ModificationDate</key>
+	<string>2013-07-28 23:08:05 +0000</string>
+	<key>Modifier</key>
+	<string>Jay Kreps</string>
+	<key>NotesVisible</key>
+	<string>NO</string>
+	<key>Orientation</key>
+	<integer>2</integer>
+	<key>OriginVisible</key>
+	<string>NO</string>
+	<key>PageBreaks</key>
+	<string>YES</string>
+	<key>PrintInfo</key>
+	<dict>
+		<key>NSBottomMargin</key>
+		<array>
+			<string>float</string>
+			<string>41</string>
+		</array>
+		<key>NSHorizonalPagination</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAhOU051bWJlcgCEhAdOU1ZhbHVlAISECE5TT2JqZWN0AIWEASqEhAFxlwCG</string>
+		</array>
+		<key>NSLeftMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSPaperSize</key>
+		<array>
+			<string>size</string>
+			<string>{612.00002479553223, 792}</string>
+		</array>
+		<key>NSPrintReverseOrientation</key>
+		<array>
+			<string>int</string>
+			<string>0</string>
+		</array>
+		<key>NSRightMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSTopMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+	</dict>
+	<key>PrintOnePage</key>
+	<false/>
+	<key>ReadOnly</key>
+	<string>NO</string>
+	<key>RowAlign</key>
+	<integer>1</integer>
+	<key>RowSpacing</key>
+	<real>36</real>
+	<key>SheetTitle</key>
+	<string>Canvas 1</string>
+	<key>SmartAlignmentGuidesActive</key>
+	<string>YES</string>
+	<key>SmartDistanceGuidesActive</key>
+	<string>YES</string>
+	<key>UniqueID</key>
+	<integer>1</integer>
+	<key>UseEntirePage</key>
+	<false/>
+	<key>VPages</key>
+	<integer>1</integer>
+	<key>WindowInfo</key>
+	<dict>
+		<key>CurrentSheet</key>
+		<integer>0</integer>
+		<key>ExpandedCanvases</key>
+		<array>
+			<dict>
+				<key>name</key>
+				<string>Canvas 1</string>
+			</dict>
+		</array>
+		<key>Frame</key>
+		<string>{{424, 6}, {711, 872}}</string>
+		<key>ListView</key>
+		<true/>
+		<key>OutlineWidth</key>
+		<integer>142</integer>
+		<key>RightSidebar</key>
+		<false/>
+		<key>ShowRuler</key>
+		<true/>
+		<key>Sidebar</key>
+		<true/>
+		<key>SidebarWidth</key>
+		<integer>120</integer>
+		<key>VisibleRegion</key>
+		<string>{{0, 0}, {576, 733}}</string>
+		<key>Zoom</key>
+		<real>1</real>
+		<key>ZoomValues</key>
+		<array>
+			<array>
+				<string>Canvas 1</string>
+				<real>1</real>
+				<real>1</real>
+			</array>
+		</array>
+	</dict>
+</dict>
+</plist>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/dag.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/dag.png b/docs/img/0.7.0/learn/documentation/introduction/dag.png
new file mode 100644
index 0000000..bda85b2
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/dag.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/group-by-example.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/group-by-example.png b/docs/img/0.7.0/learn/documentation/introduction/group-by-example.png
new file mode 100644
index 0000000..1acd355
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/group-by-example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/job.graffle
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/job.graffle b/docs/img/0.7.0/learn/documentation/introduction/job.graffle
new file mode 100644
index 0000000..2c5a994
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/introduction/job.graffle
@@ -0,0 +1,512 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN" "http://www.apple.com/DTDs/PropertyList-1.0.dtd">
+<plist version="1.0">
+<dict>
+	<key>ActiveLayerIndex</key>
+	<integer>0</integer>
+	<key>ApplicationVersion</key>
+	<array>
+		<string>com.omnigroup.OmniGrafflePro.MacAppStore</string>
+		<string>139.18</string>
+	</array>
+	<key>AutoAdjust</key>
+	<true/>
+	<key>BackgroundGraphic</key>
+	<dict>
+		<key>Bounds</key>
+		<string>{{0, 0}, {576.00002479553223, 733}}</string>
+		<key>Class</key>
+		<string>SolidGraphic</string>
+		<key>ID</key>
+		<integer>2</integer>
+		<key>Style</key>
+		<dict>
+			<key>shadow</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+			<key>stroke</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+		</dict>
+	</dict>
+	<key>BaseZoom</key>
+	<integer>0</integer>
+	<key>CanvasOrigin</key>
+	<string>{0, 0}</string>
+	<key>ColumnAlign</key>
+	<integer>1</integer>
+	<key>ColumnSpacing</key>
+	<real>36</real>
+	<key>CreationDate</key>
+	<string>2013-07-28 22:09:17 +0000</string>
+	<key>Creator</key>
+	<string>Jay Kreps</string>
+	<key>DisplayScale</key>
+	<string>1 0/72 in = 1 0/72 in</string>
+	<key>GraphDocumentVersion</key>
+	<integer>8</integer>
+	<key>GraphicsList</key>
+	<array>
+		<dict>
+			<key>Bounds</key>
+			<string>{{41, 144}, {81, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>36</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 Ouput Streams}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{47, 21}, {75, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>35</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs24 \cf0 Input Streams}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>34</integer>
+			<key>Points</key>
+			<array>
+				<string>{100.76664679221969, 109}</string>
+				<string>{100.76665277122837, 143}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>32</integer>
+			<key>Points</key>
+			<array>
+				<string>{59.5, 108}</string>
+				<string>{59.500005979008677, 142}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>19</integer>
+				<key>Info</key>
+				<integer>3</integer>
+			</dict>
+			<key>ID</key>
+			<integer>23</integer>
+			<key>Points</key>
+			<array>
+				<string>{99.5, 38}</string>
+				<string>{100.00000220537189, 71}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>19</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>22</integer>
+			<key>Points</key>
+			<array>
+				<string>{81.5, 37}</string>
+				<string>{81.500001470248037, 71}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>19</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+			<key>ID</key>
+			<integer>21</integer>
+			<key>Points</key>
+			<array>
+				<string>{63.5, 37}</string>
+				<string>{63.000000275671482, 71}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{44.5, 71}, {74, 37}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>19</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{-0.59628479784302701, -1.1925696134567261}</string>
+				<string>{1.9868216701487083e-08, -1.3333333730697632}</string>
+				<string>{0.59628487781105233, -1.1925696134567261}</string>
+				<string>{1.1925696134567272, -0.59628480672836304}</string>
+				<string>{1.3333333730697643, 1.5894572413799324e-07}</string>
+				<string>{1.1925696134567272, 0.59628473564567486}</string>
+				<string>{0.59628465308492307, 1.1925697326660156}</string>
+				<string>{1.1842379282265398e-15, 1.3333333730697632}</string>
+				<string>{-0.5962849488937394, 1.1925696134567261}</string>
+				<string>{-1.1925697326660152, 0.5962844398368361}</string>
+				<string>{-1.3333333730697625, -6.3578289655197295e-07}</string>
+				<string>{-1.1925696134567256, -0.59628480672836304}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs28 \cf0 Samza \
+Job}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+	</array>
+	<key>GridInfo</key>
+	<dict/>
+	<key>GuidesLocked</key>
+	<string>NO</string>
+	<key>GuidesVisible</key>
+	<string>YES</string>
+	<key>HPages</key>
+	<integer>1</integer>
+	<key>ImageCounter</key>
+	<integer>1</integer>
+	<key>KeepToScale</key>
+	<false/>
+	<key>Layers</key>
+	<array>
+		<dict>
+			<key>Lock</key>
+			<string>NO</string>
+			<key>Name</key>
+			<string>Layer 1</string>
+			<key>Print</key>
+			<string>YES</string>
+			<key>View</key>
+			<string>YES</string>
+		</dict>
+	</array>
+	<key>LayoutInfo</key>
+	<dict>
+		<key>Animate</key>
+		<string>NO</string>
+		<key>circoMinDist</key>
+		<real>18</real>
+		<key>circoSeparation</key>
+		<real>0.0</real>
+		<key>layoutEngine</key>
+		<string>dot</string>
+		<key>neatoSeparation</key>
+		<real>0.0</real>
+		<key>twopiSeparation</key>
+		<real>0.0</real>
+	</dict>
+	<key>LinksVisible</key>
+	<string>NO</string>
+	<key>MagnetsVisible</key>
+	<string>NO</string>
+	<key>MasterSheets</key>
+	<array/>
+	<key>ModificationDate</key>
+	<string>2013-07-28 22:21:28 +0000</string>
+	<key>Modifier</key>
+	<string>Jay Kreps</string>
+	<key>NotesVisible</key>
+	<string>NO</string>
+	<key>Orientation</key>
+	<integer>2</integer>
+	<key>OriginVisible</key>
+	<string>NO</string>
+	<key>PageBreaks</key>
+	<string>YES</string>
+	<key>PrintInfo</key>
+	<dict>
+		<key>NSBottomMargin</key>
+		<array>
+			<string>float</string>
+			<string>41</string>
+		</array>
+		<key>NSHorizonalPagination</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAhOU051bWJlcgCEhAdOU1ZhbHVlAISECE5TT2JqZWN0AIWEASqEhAFxlwCG</string>
+		</array>
+		<key>NSLeftMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSPaperSize</key>
+		<array>
+			<string>size</string>
+			<string>{612.00002479553223, 792}</string>
+		</array>
+		<key>NSPrintReverseOrientation</key>
+		<array>
+			<string>int</string>
+			<string>0</string>
+		</array>
+		<key>NSRightMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSTopMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+	</dict>
+	<key>PrintOnePage</key>
+	<false/>
+	<key>ReadOnly</key>
+	<string>NO</string>
+	<key>RowAlign</key>
+	<integer>1</integer>
+	<key>RowSpacing</key>
+	<real>36</real>
+	<key>SheetTitle</key>
+	<string>Canvas 1</string>
+	<key>SmartAlignmentGuidesActive</key>
+	<string>YES</string>
+	<key>SmartDistanceGuidesActive</key>
+	<string>YES</string>
+	<key>UniqueID</key>
+	<integer>1</integer>
+	<key>UseEntirePage</key>
+	<false/>
+	<key>VPages</key>
+	<integer>1</integer>
+	<key>WindowInfo</key>
+	<dict>
+		<key>CurrentSheet</key>
+		<integer>0</integer>
+		<key>ExpandedCanvases</key>
+		<array>
+			<dict>
+				<key>name</key>
+				<string>Canvas 1</string>
+			</dict>
+		</array>
+		<key>Frame</key>
+		<string>{{364, 6}, {711, 872}}</string>
+		<key>ListView</key>
+		<true/>
+		<key>OutlineWidth</key>
+		<integer>142</integer>
+		<key>RightSidebar</key>
+		<false/>
+		<key>ShowRuler</key>
+		<true/>
+		<key>Sidebar</key>
+		<true/>
+		<key>SidebarWidth</key>
+		<integer>120</integer>
+		<key>VisibleRegion</key>
+		<string>{{0, 0}, {576, 733}}</string>
+		<key>Zoom</key>
+		<real>1</real>
+		<key>ZoomValues</key>
+		<array>
+			<array>
+				<string>Canvas 1</string>
+				<real>1</real>
+				<real>1</real>
+			</array>
+		</array>
+	</dict>
+</dict>
+</plist>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/job.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/job.png b/docs/img/0.7.0/learn/documentation/introduction/job.png
new file mode 100644
index 0000000..4a90b8c
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/job.png differ


[11/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/comparisons/mupd8.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/comparisons/mupd8.md b/docs/learn/documentation/0.7.0/comparisons/mupd8.md
new file mode 100644
index 0000000..bb0d5a1
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/comparisons/mupd8.md
@@ -0,0 +1,72 @@
+---
+layout: page
+title: MUPD8
+---
+
+*People generally want to know how similar systems compare. We've done our best to fairly contrast the feature sets of Samza with other systems. But we aren't experts in these frameworks, and we are, of course, totally biased. If we have goofed anything let us know and we will correct it.*
+
+### Durability
+
+MUPD8 makes no durability or delivery guarantees. Within MUPD8, stream processor tasks receive messages at most once. Samza uses Kafka for messaging, which guarantees message delivery.
+
+### Ordering
+
+As with durability, developers would ideally like their stream processors to receive messages in exactly the order that they were written.
+
+We don't entirely follow MUPD8's description of their ordering guarantees, but it seems to guarantee that all messages will be processed in the order in which they are written to MUPD8 queues, which is comparable to Kafka and Samza's guarantee.
+
+### Buffering
+
+A critical issue for handling large data flows is handling back pressure when one downstream processing stage gets slow.
+
+MUPD8 buffers messages in an in-memory queue when passing messages between two MUPD8 tasks. When a queue fills up, developers are left to either drop the messages on the floor, log the messages to local disk, or block until the queue frees up. All of these options are sub-optimal. Dropping messages destroys durability guarantees. Blocking your stream processor can result in back pressure, where the slowest processor blocks all upstream processors, which in turn block their upstream processors, until the whole system comes to a grinding hault. Logging to local disk is the most reasonable, but when a fault occurs, those messages will be lost on failover.
+
+By adopting Kafka's broker as a remote buffer, Samza solves all of these problems. It doesn't need to block because consumers and producers are decoupled using Kafka's brokers' disks as async buffers. Messages shouldn't be dropped because Kafkas's 0.8 brokers should be highly available. In the event of a failure, when a Samza job resumes on another system, its input and output are not lost because it's stored remotely on replicated Kafka brokers.
+
+### State Management
+
+Stream processors frequently will accrue state as they process messages. For example, they might be incrementing a counter when a certain type of message is seen. They might also be storing messages in memory while trying to join them with messages from another stream (e.g. ad impressions vs. ad clicks). A design decision that needs to be made is how (if at all) to handle this in-memory state in situations where a failure occurs.
+
+MUPD8 uses a write back caching strategy to manage in-memory state that is periodically written back to Cassandra.
+
+Samza maintains state locally with the task. This allows state larger than will fit in memory. State is persisted to an output stream for recovery purposes should the task fail. In the long run we believe this design will be better suited to strong fault tolerance semantics as the change log captures the evolution of state allowing consistent restore of a task to a consistent point of time.
+
+### Deployment and execution
+
+MUPD8 includes a custom execution framework. The functionality that this framework supports in terms of users and resource limits isn't clear to us.
+
+Samza simply leverages YARN to deploy user code, and execute it in a distributed environment.
+
+### Fault Tolerance
+
+What should a stream processing system do when a machine or processor fails?
+
+MUPD8 uses its custom rolled equivalent to YARN to manage fault tolerance. When a stream processor is unable to send a message to a downstream processor, it notifies MUPD8's coordinator, and all other machines are notified. The machines then send all messages to a new machine based on the key hash that's used. Messages and state can both be lost when this happens.
+
+Samza uses YARN to manage fault tolerance. YARN will detect when nodes or Samza tasks fail, and will notify Samza's [ApplicationMaster](../yarn/application-master.html). At that point, it's up to Samza to decide what to do. Generally, this means re-starting the task on another machine. Since messages are persisted to Kafka brokers remotely, and there are no in-memory queues, no messages should be lost unless the processors are using async Kafka producers.
+
+### Workflow
+
+Sometimes more than one job or processing stage is needed to accomplish something. This is the case where you wish to re-partition a stream, for example. MUPD8 has a custom workflow system setup to define how to execute multiple jobs at once, and how to feed stream data from one into the other.
+
+Samza makes the individual jobs the level of granularity of execution. Jobs communicate via named input and output streams. This implicitly defines a data flow graph between all running jobs. We chose this model to enable data flow graphs with processing stages owned by different engineers on different teams working in different code bases without the need to wire everything together into a single topology.
+
+This was motivated by our experience with Hadoop where the data flow between jobs is implicitly wired together by their input and output directories. We have had good experience making this decentralized model work well.
+
+### Memory
+
+MUPD8 executes all of its map/update processors inside a single JVM, using threads. This should shrink the memory footprint of a stream processor by amortizing JVM overhead across the number of processors currently being executed.
+
+Samza tends to use more memory since it has distinct JVMs for each stream processor container ([TaskRunner](../container/task-runner.html)), rather than, running multiple stream processors in the same JVM, which is what MUPD8 does. The benefit of having separate processes, however, is isolation.
+
+### Isolation
+
+MUPD8 provides no stream processor isolation. A single badly behaved stream processor can bring down all processors on the node.
+
+Samza uses process level isolation between stream processor tasks. This is also the approach that Hadoop takes. We can enforce strict per-process memory footprints. In addition, Samza supports CPU limits when used with YARN CGroups. As YARN CGroup maturity progresses, the possibility to support disk and network CGroup limits should become available as well.
+
+### Further Reading
+
+The MUPD8 team has published a very good [paper](http://vldb.org/pvldb/vol5/p1814_wanglam_vldb2012.pdf) on the design of their system.
+
+## [Storm &raquo;](storm.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/comparisons/storm.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/comparisons/storm.md b/docs/learn/documentation/0.7.0/comparisons/storm.md
new file mode 100644
index 0000000..0372d7c
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/comparisons/storm.md
@@ -0,0 +1,94 @@
+---
+layout: page
+title: Storm
+---
+
+*People generally want to know how similar systems compare. We've done our best to fairly contrast the feature sets of Samza with other systems. But we aren't experts in these frameworks, and we are, of course, totally biased. If we have goofed anything let us know and we will correct it.*
+
+[Storm](http://storm-project.net/) and Samza are fairly similar. Both systems provide many of the same features: a partitioned stream model, a distributed execution environment, an API for stream processing, fault tolerance, Kafka integration, etc.
+
+### Ordering and Guarantees
+
+Storm has more conceptual building blocks than Samza. "Spouts" in Storm are similar to Streams in Samza, and Samza does not have an equivalent of their transient zeromq communication.
+
+There are also several approaches to handling delivery guarantees.
+
+The primary approach is implemented by keeping a record of all emitted records in memory until they are acknowledged by all elements of a particular processing graph. In this mode messages that timeout are re-emitted. This seems to imply that messages can be processed out of order. This mechanism requires some co-operation from the user code which must maintain the ancestry of records in order to properly acknowledge its input. This is detailed in-depth on [Storm's wiki](https://github.com/nathanmarz/storm/wiki/Guaranteeing-message-processing).
+
+Out of order processing is a problem for handling keyed data. For example if you have a stream of database updates where later updates may replace earlier updates then reordering them may change the output.
+
+This mechanism also implies that individual stages may produce back pressure up the processing graph, so the graphs are probably mostly limited to a single logical function. However multiple graphs could likely be stitched together using Spouts in between to buffer.
+
+Storm offers a secondary approach to delivery guarantees called [transactional topologies](https://github.com/nathanmarz/storm/wiki/Transactional-topologies). These require an underlying system similar to Kafka that maintains strongly sequenced messages. Transactional topologies seem to be limited to a single input stream.
+
+Samza always offers guaranteed delivery and ordering of input within a stream partition. We make no guarantee of ordering between different input streams or input stream partitions. Since all stages are repayable there is no need for the user code to track its ancestry.
+
+Like Storm's transactional topologies Samza provides a unique "offset" which is a sequential integer uniquely denoting the message in that stream partition. That is the first message in a stream partition has offset 0, the second offset 1, etc. Samza always records the position of a job in its input streams as a vector of offsets for the input stream partitions it consumers.
+
+Storm has integrated these transaction ids into some of its storage abstractions to help with deduplicating updates. We have a different take on ensuring the semantics of output in the presence of failures however we have not yet implemented this.
+
+### State Management
+
+We are not aware of any state management facilities in Storm though transactional topologies have plugins for external storage to use the transaction id for deduping. In this case, Storm will manage only the metadata necessary to make a topology transactional. It's still up to the Bolt implementer to handle transaction IDs, and store state in a remote database, somewhere.
+
+Samza provides [built-in primitives](../container/state-management.html) for managing large amounts of state.
+
+### Partitioning and Parallelism
+
+Storm's [parallelism model](https://github.com/nathanmarz/storm/wiki/Understanding-the-parallelism-of-a-Storm-topology) maps fairly similar to Samza's. The biggest difference is that Samza holds only a single job per process and the process is single threaded regardless of the number of tasks it contains. Storm's more optimistic parallelism model has the advantage of taking better advantage of excess capacity on an idle machine. However this significantly complicates the resource model. In Samza since each container map exactly to a CPU core a job run in 100 containers will use 100 CPU cores. This allows us to better model the CPU usage on a machine and ensure that we don't see uneven performance based on the other tasks that happen to be collocated on that machine. 
+
+Storm supports "dynamic rebalancing", which means adding more threads or processes to a topology without restarting the topology or cluster. This is a convenient feature, especially for during development. We haven't added this yet as philosophically we feel that these kind of changes should go through a normal configuration management process (i.e. version control, notification, etc) as they impact production performance. In other words the jobs + configs should fully recreate the state of the cluster.
+
+### Deployment &amp; Execution
+
+A Storm cluster is composed of a series of nodes running a "Supervisor" daemon. The supervisor daemons talk to a single master node running a daemon called "Nimbus". The Nimbus daemon is responsible for assigning work and managing resources in the cluster. See Storm's [Tutorial](https://github.com/nathanmarz/storm/wiki/Tutorial) page for details. This is quite similar to YARN; though YARN is a bit more fully featured and intended to be multi-framework, Nimbus is better integrated with Storm.
+
+Yahoo! has also released [Storm-YARN](https://github.com/yahoo/storm-yarn). As described in [this Yahoo! blog post](http://developer.yahoo.com/blogs/ydn/storm-yarn-released-open-source-143745133.html), Storm-YARN is a wrapper that starts a single Storm cluster (complete with Nimbus, and Supervisors) inside a YARN grid.
+
+Anyone familiar with YARN will recognize the similarity between Storm's "Nimbus" daemon, and YARN's ResourceManager, and Storm's "Supervisor" daemon, and YARN's Node Managers. Rather than writing its own resource management framework, or running a second one inside of YARN, Samza simply uses YARN directly, as a first-class citizen in the Samza ecosystem. YARN is stable, well adopted, fully-featured, and inter-operable with Hadoop. It also provides a bunch of nice features like Security, CGroup process isolation, etc.
+
+### Language Support
+
+Storm is written in Java and Clojure but has good support for non-JVM languages. It follows a model similar to MapReduce Streaming by piping input and output streams fed to externally managed processes.
+
+On top of this, Storm provides [Trident](https://github.com/nathanmarz/storm/wiki/Trident-tutorial), a DSL that's meant to make writing Storm topologies easier.
+
+Samza is built with language support in mind, but currently only supports JVM languages.
+
+### Workflow
+
+Storm provides modeling of "Topologies" (a processing graph of multiple stages) [in code](https://github.com/nathanmarz/storm/wiki/Tutorial). This manual wiring together of the flow can serve as nice documentation of the processing flow.
+
+Each job in a Samza graph is an independent entity that communicates with other jobs through a named stream rather than manually wiring them together. All the jobs on a cluster comprise a single (potentially disconnected) data flow graph. Each job can be stopped or started independently and there is no code coupling between jobs.
+
+### Maturity
+
+We can't speak to Storm's maturity, but it has an [impressive amount of adopters](https://github.com/nathanmarz/storm/wiki/Powered-By), a strong feature set, and seems to be under active development. It integrates well with many common messaging systems (RabbitMQ, Kesrel, Kafka, etc).
+
+Samza is pretty immature, though it builds on solid components. YARN is fairly new, but is already being run on 3000+ node clusters at Yahoo!, and the project is under active development by both [Hortonworks](http://hortonworks.com/) and [Cloudera](http://www.cloudera.com/content/cloudera/en/home.html). Kafka has a strong [powered by](https://cwiki.apache.org/KAFKA/powered-by.html) page, and has seen its share of adoption, recently. It's also frequently used with Storm. Samza is a brand new project that is in use at LinkedIn. Our hope is that others will find it useful, and adopt it as well.
+
+### Buffering &amp; Latency
+
+Within a single topology, Storm has producers and consumers, but no broker (to use Kafka's terminology). This design decision leads to a number of interesting properties.
+
+Since Storm uses ZeroMQ without intermediate brokers, the transmission of messages from one Bolt to another is extremely low latency. It's just a network hop.
+
+On the flip side, when a Bolt is trying to send messages using ZeroMQ, and the consumer can't read them fast enough, the ZeroMQ buffer in the producer's process begins to fill up with messages. When it becomes full, you have the option to drop them, log to local disk, or block until space becomes available again. These options are outlined in the [MUPD8 comparison](mupd8) page, as well, and none of them are ideal. This style of stream processing runs the risk of completely grinding to a halt (or dropping messages) if a single Bolt has a throughput issue. This problem is commonly known as back pressure. When back pressure occurs, Storm essentially offloads the problem to the Spout implementation. In cases where the Spout can't handle large volumes of back-logged messages, the same problem occurs. In systems like Kafka, where large volumes of backlogged messages are supported, the entire topology just reads messages from the spout at a lower rate.
+
+A lack of a broker between bolts also adds complexity when trying to deal with fault tolerance and messaging semantics. Storm has a very well written page on [Transactional Topologies](https://github.com/nathanmarz/storm/wiki/Transactional-topologies) that describes this problem, and Storm's solution, in depth.
+
+Samza takes a different approach to buffering. We buffer to disk at every hop between a StreamTask. This decision, and its trade-offs, are described in detail on the [Comparison Introduction](introduction.html) page's "stream model" section. This design decision lets us cheat a little bit, when it comes to things like durability guarantees, and exactly once messaging semantics, but it comes at the price of increased latency, since everything must be written to disk in Kafka.
+
+### Isolation
+
+Storm provides standard UNIX process-level isolation. Your topology can impact another topology's performance (or vice-versa) if too much CPU, disk, network, or memory is used.
+
+Samza relies on YARN to provide resource-level isolation. Currently, YARN provides explicit controls for memory and CPU limits (through [CGroups](../yarn/isolation.html)), and both have been used successfully with Samza. No isolation for disk or network is provided by YARN at this time.
+
+### Data Model
+
+Storm models all messages as "Tuples" with a defined data model but pluggable serialization.
+
+Samza's serialization and data model are both pluggable. We are not terribly opinionated about which approach is best.
+
+## [API Overview &raquo;](../api/overview.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/checkpointing.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/checkpointing.md b/docs/learn/documentation/0.7.0/container/checkpointing.md
new file mode 100644
index 0000000..b0cd04f
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/checkpointing.md
@@ -0,0 +1,45 @@
+---
+layout: page
+title: Checkpointing
+---
+
+On the [Streams](streams.html) page, on important detail was glossed over. When a TaskRunner instantiates a StreamConsumer for an input stream/partition pair, how does the TaskRunner know where in the stream to start reading messages. If you recall, Kafka has the concept of an offset, which defines a specific location in a topic/partition pair. The idea is that an offset can be used to reference a specific point in a stream/partition pair. When you read messages from Kafka, you can supply an offset to specify at which point you'd like to read from. After you read, you increment your offset, and get the next message.
+
+![diagram](/img/0.7.0/learn/documentation/container/checkpointing.png)
+
+This diagram looks the same as on the [Streams](streams.html) page, except that there are black lines at different points in each input stream/partition pair. These lines represent the current offset for each stream consumer. As the stream consumer reads, the offset increases, and moves closer to the "head" of the stream. The diagram also illustrates that the offsets might be staggered, such that some offsets are farther along in their stream/partition than others.
+
+If a StreamConsumer is reading messages for a TaskRunner, and the TaskRunner stops for some reason (due to hardware failure, re-deployment, or whatever), the StreamConsumer should start where it left off when the TaskRunner starts back up again. We're able to do this because the Kafka broker is buffering messages on a remote server (the broker). Since the messages are available when we come back, we can just start from our last offset, and continue moving forward, without losing data.
+
+The TaskRunner supports this ability using something called a CheckpointManager.
+
+```
+public interface CheckpointManager {
+  public void writeCheckpoint(Partition partition, Checkpoint checkpoint);
+
+  public Checkpoint readLastCheckpoint(Partition partition);
+
+  public void close();
+}
+
+public class Checkpoint {
+  private final Map<String, String> offsets;
+  ...
+}
+```
+
+As you can see, the checkpoint manager provides a way to write out checkpoints for a given partition. Right now, the checkpoint contains a map. The map's keys are input stream names, and the map's values are each input stream's offset. Each checkpoint is managed per-partition. For example, if you have page-view-event and service-metric-event defined as streams in your Samza job's configuration file, the TaskRunner would supply a checkpoint with two keys in each checkpoint offset map (one for page-view-event and the other for service-metric-event).
+
+Samza provides two checkpoint managers: FileSystemCheckpointManager and KafkaCheckpointManager. The KafkaCheckpointManager is what you generally want to use. The way that KafkaCheckpointManager works is as follows: it writes checkpoint messages for your Samza job to a special Kafka topic. This topic's name is \_\_samza\_checkpoint\_your-job-name. For example, if you had a Samza job called "my-first-job", the Kafka topic would be called \_\_samza\_checkpoint\_my-first-job. This Kafka topic is partitioned identically to your Samza job's partition count. If your Samza job has 10 partitions, the checkpoint topic for your Samza job will also have 10 partitions. Every time that the TaskRunner calls writeCheckpoint, a checkpoint message will be sent to the partition that corresponds with the partition for the checkpoint that the TaskRunner wishes to write.
+
+![diagram](/img/0.7.0/learn/documentation/container/checkpointing-2.png)
+
+When the TaskRunner starts for the first time, the offset behavior of the StreamConsumers is undefined. If the system for the StreamConsumer is Kafka,, we fall back to the autooffest.reset setting. If the autooffset.reset is set to "largest", we start reading messages from the head of the stream; if it's set to "smallest", we read from the tail. If it's undefined, the TaskRunner will fail.
+
+The TaskRunner calls writeCheckpoint at a windowed interval (e.g. every 10 seconds). If the TaskRunner fails, and restarts, it simply calls readLastCheckpoint for each partition. In the case of the KafkaCheckpointManager, this readLastCheckpoint method will read the last message that was written to the checkpoint topic for each partition in the job. One edge case to consider is that StreamConsumers might have read messages from an offset that hasn't yet been checkpointed. In such a case, when the TaskRunner reads the last checkpoint for each partition, the offsets might be farther back in the stream. When this happens, your StreamTask could get duplicate messages (i.e. it saw message X, failed, restarted at an offset prior to message X, and then reads message X again). Thus, Samza currently provides at least once messaging. You might get duplicates. Caveat emptor.
+
+<!-- TODO Add a link to the fault tolerance SEP when one exists -->
+
+*Note that there are design proposals in the works to give exactly once messaging.*
+
+## [State Management &raquo;](state-management.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/event-loop.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/event-loop.md b/docs/learn/documentation/0.7.0/container/event-loop.md
new file mode 100644
index 0000000..4069ef0
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/event-loop.md
@@ -0,0 +1,61 @@
+---
+layout: page
+title: Event Loop
+---
+
+The event loop is the [TaskRunner](task-runner.html)'s single thread that is in charge of [reading](streams.html), [writing](streams.html), [metrics flushing](metrics.html), [checkpointing](checkpointing.html), and [windowing](windowing.html). It's the code that puts all of this stuff together. Each StreamConsumer reads messages on its own thread, but writes messages into a centralized message queue. The TaskRunner uses this queue to funnel all of the messages into the event loop. Here's how the event loop works:
+
+1. Take a message from the incoming message queue (the queue that the StreamConsumers are putting their messages)
+2. Give the message to the appropriate StreamTask by calling process() on it
+3. Send any StreamTask output from the process() call to the appropriate StreamProducers
+4. Call window() on the StreamTask if it implements WindowableTask, and the window time has expired
+5. Send any StreamTask output from the window() call to the appropriate StreamProducers
+6. Write checkpoints for any partitions that are past the defined checkpoint commit interval
+
+The TaskRunner does this, in a loop, until it is shutdown.
+
+### Lifecycle Listeners
+
+Sometimes, it's useful to receive notifications when a specific event happens in the TaskRunner. For example, you might want to reset some context in the container whenever a new message arrives. To accomplish this, Samza provides a TaskLifecycleListener interface, that can be wired into the TaskRunner through configuration.
+
+```
+/**
+ * Used to get before/after notifications before initializing/closing all tasks
+ * in a given container (JVM/process).
+ */
+public interface TaskLifecycleListener {
+  /**
+   * Called before all tasks in TaskRunner are initialized.
+   */
+  void beforeInit(Config config, TaskContext context);
+
+  /**
+   * Called after all tasks in TaskRunner are initialized.
+   */
+  void afterInit(Config config, TaskContext context);
+
+  /**
+   * Called before a message is processed by a task.
+   */
+  void beforeProcess(MessageEnvelope envelope, Config config, TaskContext context);
+
+  /**
+   * Called after a message is processed by a task.
+   */
+  void afterProcess(MessageEnvelope envelope, Config config, TaskContext context);
+
+  /**
+   * Called before all tasks in TaskRunner are closed.
+   */
+  void beforeClose(Config config, TaskContext context);
+
+  /**
+   * Called after all tasks in TaskRunner are closed.
+   */
+  void afterClose(Config config, TaskContext context);
+}
+```
+
+The TaskRunner will notify any lifecycle listeners whenever one of these events occurs. Usually, you don't really need to worry about lifecycle, but it's there if you need it.
+
+## [JMX &raquo;](jmx.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/index.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/index.md b/docs/learn/documentation/0.7.0/container/index.md
new file mode 100644
index 0000000..17751de
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/index.md
@@ -0,0 +1,18 @@
+---
+layout: page
+title: Container
+---
+
+The API section shows how a Samza StreamTask is written. To execute a StreamTask, Samza has a container that wraps around your StreamTask. The Samza container manages:
+
+* Metrics
+* Configuration
+* Lifecycle
+* Checkpointing
+* State management
+* Serialization
+* Data transport
+
+This container is called a TaskRunner. Read on to learn more about Samza's TaskRunner.
+
+## [JobRunner &raquo;](job-runner.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/jmx.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/jmx.md b/docs/learn/documentation/0.7.0/container/jmx.md
new file mode 100644
index 0000000..a9fcc77
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/jmx.md
@@ -0,0 +1,13 @@
+---
+layout: page
+title: JMX
+---
+
+The Samza TaskRunner (and YARN Application Master) will turn on JMX using a randomly selected port, since Samza is meant to be run in a distributed environment, and it's unknown which ports will be available prior to runtime. The port will be output in the TaskRunner's logs with a line like this:
+
+    2013-07-05 20:42:36 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are Chriss-MacBook-Pro.local
+    2013-07-05 20:42:36 JmxServer [INFO] Started JmxServer port=64905 url=service:jmx:rmi:///jndi/rmi://Chriss-MacBook-Pro.local:64905/jmxrmi
+
+Any metrics that are registered in the TaskRunner will be visible through JMX. To toggle JMX, see the [Configuration](../jobs/configuration.html) section.
+
+## [JobRunner &raquo;](../jobs/job-runner.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/metrics.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/metrics.md b/docs/learn/documentation/0.7.0/container/metrics.md
new file mode 100644
index 0000000..4a3e403
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/metrics.md
@@ -0,0 +1,50 @@
+---
+layout: page
+title: Metrics
+---
+
+Samza also provides a metrics library that the TaskRunner uses. It allows a StreamTask to create counters and gauges. The TaskRunner then writes those metrics to metrics infrastructure through a MetricsReporter implementation.
+
+```
+public class MyJavaStreamerTask implements StreamTask, InitableTask {
+  private static final Counter messageCount;
+
+  public void init(Config config, TaskContextPartition context) {
+    this.messageCount = context.getMetricsRegistry().newCounter(MyJavaStreamerTask.class.toString(), "MessageCount");
+  }
+
+  @Override
+  public void process(MessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    System.out.println(envelope.getMessage().toString());
+    messageCount.inc();
+  }
+}
+```
+
+Samza's metrics design is very similar to Coda Hale's [metrics](https://github.com/codahale/metrics) library. It has two important interfaces:
+
+```
+public interface MetricsRegistry {
+  Counter newCounter(String group, String name);
+
+  <T> Gauge<T> newGauge(String group, String name, T value);
+}
+
+public interface MetricsReporter {
+  void report(MessageCollector collector, ReadableMetricsRegistry registry, long currentTimeMillis, Partition partition);
+}
+```
+
+### MetricsRegistry
+
+When the TaskRunner starts up, as with StreamTask instantiation, it creates a MetricsRegistry for every partition in the Samza job.
+
+![diagram](/img/0.7.0/learn/documentation/container/metrics.png)
+
+The TaskRunner, itself, also gets a MetricsRegistry that it can use to create counters and gauges. It uses this registry to measure a lot of relevant metrics for itself.
+
+### MetricsReporter
+
+The other important interface is the MetricsReporter. The TaskRunner uses MetricsReporter implementations to send its MetricsRegistry counters and gauges to whatever metrics infrastructure the reporter uses. A Samza job's configuration determines which MetricsReporters the TaskRunner will use. Out of the box, Samza comes with a MetricsSnapshotReporter that sends JSON metrics messages to a Kafka topic.
+
+## [Windowing &raquo;](windowing.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/state-management.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/state-management.md b/docs/learn/documentation/0.7.0/container/state-management.md
new file mode 100644
index 0000000..0e3b9b1
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/state-management.md
@@ -0,0 +1,115 @@
+---
+layout: page
+title: State Management
+---
+
+Samza allows tasks to maintain persistent, mutable state that is physically co-located with each task. The state is highly available: in the event of a task failure it will not be lost when the task fails over to another machine.
+
+A key-value store implementation is provided out of the box that covers many use cases. Other store implementations can be plugged in for different types of storage.
+
+State is naturally partitioned with the tasks, with one store per task. When there is a backing changelog, the stream will also be co-partitioned with the tasks. Possible extensions to handle non-partitioned state (i.e. a global lookup dictionary) are discussed at the end.
+
+Restoring state can be done either by having a dedicated stream that captures the changes to the local store, or by rebuilding the state off the input streams.
+
+### Use Cases
+
+We have a few use-cases in mind for this functionality.
+
+#### Windowed Aggregation
+
+Example: Counting member page views by hour
+
+Implementation: The stream is partitioned by the aggregation key (member\_id). Each new input record would cause the job to retrieve and update the aggregate (the page view count). When the window is complete (i.e. the hour is over), the job outputs the current stored aggregate value.
+
+####Table-Table Join
+
+Example: Join profile to user\_settings by member\_id and emit the joined stream
+
+Implementation: The job subscribes to the change stream for profile and for user\_settings both partitioned by member\_id. The job keeps a local store containing both the profile and settings data. When a record comes in from either profile or settings, the job looks up the value for that member and updates the appropriate section (either profile or settings). The changelog for the local store can be used as the output stream if the desired output stream is simply the join of the two inputs.
+
+#### Table-Stream Join
+
+Example: Join member geo region to page view data
+
+Implementation: The job subscribes to the profile stream (for geo) and page views stream, both partitioned by member\_id. It keeps a local store of member\_id => geo that it updates off the profile feed. When a page view arrives it does a lookup in this store to join on the geo data.
+
+#### Stream-Stream Join
+
+Example: Join ad clicks to ad impressions by some shared key
+
+Implementation: Partition ad click and ad impression by the join key. Keep a store of unmatched clicks and unmatched impressions. When a click comes in try to find its matching impression in the impression store, and vice versa when an impression comes in check the click store. If a match is found emit the joined pair and delete the entry. If no match is found store the event to wait for a match. Since this is presumably a left outer join (i.e. every click has a corresponding impression but not vice versa) we will periodically scan the impression table and delete old impressions for which no click arrived.
+
+#### More
+
+Of course there are infinite variations on joins and aggregations, but most amount to essentially variations on the above.
+
+### Usage
+
+To declare a new store for usage you add the following to your job config:
+
+    # Use the key-value store implementation for 
+    stores.my-store.factory=samza.storage.kv.KeyValueStorageEngineFactory
+    # Log changes to the store to a stream
+    stores.my-store.changelog=my-stream-name
+    # The serialization format to use
+    stores.my-store.serde=string
+    # The system to use for the changelog
+    stores.my-store.system=kafka
+
+Example code:
+
+    public class MyStatefulTask implements StreamTask, InitableTask {
+      private KeyValueStore<String, String> store;
+      
+      public void init(Config config, TaskContextPartition context) {
+        this.store = (KeyValueStore<String, String>) context.getStore("store");
+      }
+
+      public void process(MessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+        System.out.println("Adding " + envelope.getMessage() + " => " + envelope.getMessage() + " to the store.");
+        store.put((String) envelope.getMessage(), (String) envelope.getMessage());
+      }
+    }
+
+This shows the put() API, but KeyValueStore gives a fairly general key-value interface:
+
+    public interface KeyValueStore<K, V> {
+      V get(K key);
+      void put(K key, V value);
+      void putAll(List<Entry<K,V>> entries);
+      void delete(K key);
+      KeyValueIterator<K,V> range(K from, K to);
+      KeyValueIterator<K,V> all();
+    }
+
+### Implementing Storage Engines
+
+The above code shows usage of the key-value storage engine, but it is not too hard to implement an alternate storage engine. To do so, you implement methods to restore the contents of the store from a stream, flush any cached content on commit, and close the store:
+
+    public interface StorageEngine {
+      void restore(StreamConsumer consumer);
+      void flush();
+      void close();
+    }
+
+The user specifies the type of storage engine they want by passing in a factory for that store in their configuration.
+
+### Fault Tolerance Semantics with State
+
+Samza currently only supports at-least-once delivery guarantees. We will extend this to exact atomic semantics across outputs to multiple streams/partitions in the future.
+
+<!-- TODO add fault tolerance semantics SEP link when one exists
+The most feasible plan for exact semantics seems to me to be journalling non-deterministic decisions proposal outlined in the fault-tolerance semantics wiki. I propose we use that as a working plan.
+
+To ensure correct semantics in the presence of faults we need to ensure that the task restores to the exact state at the time of the last commit.
+
+If the task is fed off replayable inputs then it can simply replay these inputs to recreate its state.
+
+If the task has a changelog to log its state then there is the possibility that the log contains several entries beyond the last commit point. The store should only restore up to the last commit point to ensure that the state is in the correct position with respect to the inputs–the remaining changelog will then be repeated and de-duplicated as the task begins executing.
+-->
+
+### Shared State
+
+Originally we had discussed possibly allowing some facility for global lookup dictionaries that are un-partitioned; however, this does not work with our fault-tolerance semantics proposal, as the container-wide state changes out of band (effectively acting like a separate database or service). This would not work with proposed message de-duplication features since the task output is not deterministic.
+
+## [Metrics &raquo;](metrics.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/streams.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/streams.md b/docs/learn/documentation/0.7.0/container/streams.md
new file mode 100644
index 0000000..e755789
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/streams.md
@@ -0,0 +1,40 @@
+---
+layout: page
+title: Streams
+---
+
+The [TaskRunner](task-runner.html) reads and writes messages using the StreamConsumer and StreamProducer interfaces.
+
+```
+public interface StreamConsumer {
+  StreamConsumerMessageEnvelope getMessage();
+
+  void close();
+}
+
+public interface StreamConsumerMessageEnvelope {
+  ByteBuffer getMessage();
+
+  String getOffsetId();
+}
+
+public interface StreamProducer<K> {
+  void send(ByteBuffer bytes);
+
+  void send(K k, ByteBuffer bytes);
+
+  void commit();
+
+  void close();
+}
+```
+
+Out of the box, Samza supports reads and writes to Kafka (i.e. it has a KafkaStreamConsumer/KafkaStreamProducer), but the stream interfaces are pluggable, and most message bus systems can be plugged in, with some degree of support.
+
+A number of stream-related properties should be defined in your Samza job's configuration file. These properties define systems that Samza can read from, the streams on these systems, and how to serialize and deserialize the messages from the streams. For example, you might wish to read PageViewEvent from a specific Kafka cluster. The system properties in the configuration file would define how to connect to the Kafka cluster. The stream section would define PageViewEvent as an input stream. The serializer in the configuration would define the serde to use to decode PageViewEvent messages.
+
+When the TaskRunner starts up, it will use the stream-related properties in your configuration to instantiate consumers for each stream partition. For example, if your input stream is PageViewEvent, which has 12 partitions, then the TaskRunner would create twelve KafkaStreamConsumers. Each stream consumer will read ByteBuffers from one partition, deserialize the ByteBuffer to an object, and put them into a queue. This queue is what the [event loop](event-loop.html) will use to feed messages to your StreamTask instances.
+
+In the process method in StreamTask, there is a MessageCollector parameter given to use. When the TaskRunner calls process() on one of your StreamTask instances, it provides the collector. After the process() method completes, the TaskRunner takes any output messages that your StreamTask wrote to the collector, serializes the messages, and calls the send() method on the appropriate StreamProducer.
+
+## [Checkpointing &raquo;](checkpointing.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/task-runner.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/task-runner.md b/docs/learn/documentation/0.7.0/container/task-runner.md
new file mode 100644
index 0000000..2e94926
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/task-runner.md
@@ -0,0 +1,43 @@
+---
+layout: page
+title: TaskRunner
+---
+
+The TaskRunner is Samza's stream processing container. It is responsible for managing the startup, execution, and shutdown of one or more StreamTask instances.
+
+When the a TaskRunner starts up, it does the following:
+
+1. Get last checkpointed offset for each input stream/partition pair
+2. Create a "reader" thread for every input stream/partition pair
+3. Start metrics reporters to report metrics
+4. Start a checkpoint timer to save your task's input stream offsets every so often
+5. Start a window timer to trigger your StreamTask's window method, if it is defined
+6. Instantiate and initialize your StreamTask once for each input stream partition
+7. Start an event loop that takes messages from the input stream reader threads, and gives them to your StreamTasks
+8. Notify lifecycle listeners during each one of these steps
+
+Let's go over each of these items, starting in the middle, with the instantiation of a StreamTask.
+
+### Tasks and Partitions
+
+When the TaskRunner starts, it creates an instance of the StreamTask that you've written. If the StreamTask implements the InitableTask interface, the TaskRunner will also call the init() method.
+
+```
+public interface InitableTask {
+  void init(Config config, TaskContextPartition context);
+}
+```
+
+It doesn't just do this once, though. It creates the StreamTask once for each partition in your Samza job. If your Samza job has ten partitions, there will be ten instantiations of your StreamTask: one for each partition. The StreamTask instance for partition one will receive all messages for partition one, the instance for partition two will receive all messages for partition two, and so on.
+
+The number of partitions that a Samza job has is determined by the number of partitions in its input streams. If a Samza job is set up to read from a topic called PageViewEvent, which has 12 partitions, then the Samza job will have 12 partitions when it executes.
+
+![diagram](/img/0.7.0/learn/documentation/container/tasks-and-partitions.png)
+
+If a Samza job has more than one input stream, then the number of partitions for the Samza job will be the maximum number of partitions across all input streams. For example, if a Samza job is reading from PageView event, which has 12 partitions, and ServiceMetricEvent, which has 14 partitions, then the Samza job would have 14 partitions (0 through 13).
+
+When the TaskRunner's StreamConsumer threads are reading messages from each input stream partition, the messages that it receives are tagged with the partition number that it came from. Each message is fed to the StreamTask instance that corresponds to the message's partition. This design has two important properties. When a Samza job has more than one input stream, and those streams have an imbalanced number of partitions (e.g. one has 12 partitions and the other has 14), then some of your StreamTask instances will not receive messages from all streams. In the PageViewEvent/ServiceMetricEvent example, the last two StreamTask instances would only receive messages from the ServiceMetricEvent topic (partitions 12 and 13). The lower 12 instances would receive messages from both streams. If your Samza job is reading more than one input stream, you probably want all input streams to have the same number of partitions, especially if you're trying to join streams together. The second impor
 tant property is that Samza assumes that a stream's partition count will never change. No partition splitting is supported. If an input stream has N partitions, it is expected that it has had, and will always have N partitions. If you want to re-partition, you must read messages from the stream, and write them out to a new stream that has the number of partitions that you want. For example you could read messages from PageViewEvent, and write them to PageViewEventRepartition, which could have 14 partitions. If you did this, then you would achieve balance between PageViewEventRepartition and ServiceMetricEvent.
+
+This design is important because it guarantees that any state that your StreamTask keeps in memory will be isolated on a per-partition basis. For example, if you refer back to the page-view counting job we used as an example in the [Architecture](../introduction/architecture.html) section, we might have a Map&lt;Integer, Integer&gt; map that keeps track of page view counts per-member ID. If we were to have just one StreamTask per Samza job, for instance, then the member ID counts from different partitions would be inter-mingled into the same map. This inter-mingling would prevent us from moving partitions between processes or machines, which is something that we want to do with YARN. You can imagine a case where you started with one TaskRunner in a single YARN container. Your Samza job might be unable to keep up with only one container, so you ask for a second YARN container to put some of the StreamTask partitions. In such a case, how would we split the counts such that one contain
 er gets only member ID counts for the partitions in charge of? This is effectively impossible if we've inter-mingled the StreamTask's state together. This is why we isolate StreamTask instances on a per-partition basis: to make partition migration possible.
+
+## [Streams &raquo;](streams.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/container/windowing.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/windowing.md b/docs/learn/documentation/0.7.0/container/windowing.md
new file mode 100644
index 0000000..0a2e647
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/windowing.md
@@ -0,0 +1,16 @@
+---
+layout: page
+title: Windowing
+---
+
+Referring back to the, "count PageViewEvent by member ID," example in the [Architecture](../introduction/architecture.html) section, one thing that we left out was what we do with the counts. Let's say that the Samza job wants to update the member ID counts in a database once every minute. Here's how it would work. The Samza job that does the counting would keep a Map&lt;Integer, Integer&gt; in memory, which maps member IDs to page view counts. Every time a message arrives, the job would take the member ID in the PageViewEvent, and use it to increment the member ID's count in the in-memory map. Then, once a minute, the StreamTask would update the database (total_count += current_count) for every member ID in the map, and then reset the count map.
+
+Windowing is how we achieve this. If a StreamTask implements the WindowableTask interface, the TaskRunner will call the window() method on the task over a configured interval.
+
+```
+public interface WindowableTask {
+  void window(MessageCollector collector, TaskCoordinator coordinator);
+}
+```
+
+If you choose to implement the WindowableTask interface, you can use the Samza job's configuration to define how often the TaskRunner should call your window() method. In the PageViewEvent example (above), you would define it to flush every 60000 milliseconds (60 seconds).

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/index.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/index.html b/docs/learn/documentation/0.7.0/index.html
new file mode 100644
index 0000000..7806baf
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/index.html
@@ -0,0 +1,73 @@
+---
+layout: page
+title: Documentation
+---
+
+<h4>Introduction</h4>
+
+<div class="documentation-second-level">
+  <a href="introduction/background.html">Background</a><br/>
+  <a href="introduction/concepts.html">Concepts</a><br/>
+  <a href="introduction/architecture.html">Architecture</a>
+</div>
+
+<h4>Comparisons</h4>
+
+<div class="documentation-second-level">
+  <a href="comparisons/introduction.html">Introduction</a><br/>
+  <a href="comparisons/mupd8.html">MUPD8</a><br/>
+  <a href="comparisons/storm.html">Storm</a>
+<!-- TODO comparisons pages
+  <a href="comparisons/aurora.html">Aurora</a><br/>
+  <a href="comparisons/jms.html">JMS</a><br/>
+  <a href="comparisons/s4.html">S4</a><br/>
+-->
+</div>
+
+<h4>API</h4>
+
+<div class="documentation-second-level">
+  <a href="api/overview.html">Overview</a><br/>
+  <a href="api/javadocs">Javadocs</a><br/>
+</div>
+
+<h4>Container</h4>
+
+<div class="documentation-second-level">
+  <a href="container/task-runner.html">TaskRunner</a><br/>
+  <a href="container/streams.html">Streams</a><br/>
+  <a href="container/checkpointing.html">Checkpointing</a><br/>
+  <a href="container/state-management.html">State Management</a><br/>
+  <a href="container/metrics.html">Metrics</a><br/>
+  <a href="container/windowing.html">Windowing</a><br/>
+  <a href="container/event-loop.html">Event Loop</a><br/>
+  <a href="container/jmx.html">JMX</a>
+</div>
+
+<h4>Jobs</h4>
+
+<div class="documentation-second-level">
+  <a href="jobs/job-runner.html">JobRunner</a><br/>
+  <a href="jobs/configuration.html">Configuration</a><br/>
+  <a href="jobs/packaging.html">Packaging</a><br/>
+  <a href="jobs/yarn-jobs.html">YARN Jobs</a><br>
+  <a href="jobs/logging.html">Logging</a>
+</div>
+
+<h4>YARN</h4>
+
+<div class="documentation-second-level">
+  <a href="yarn/application-master.html">Application Master</a><br/>
+  <a href="yarn/isolation.html">Isolation</a>
+<!-- TODO write yarn pages
+  <a href="">Fault Tolerance</a><br/>
+  <a href="">Security</a><br/>
+-->
+</div>
+
+<h4>Operations</h4>
+
+<div class="documentation-second-level">
+  <a href="operations/security.html">Security</a><br/>
+  <a href="operations/kafka.html">Kafka</a>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/introduction/architecture.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/introduction/architecture.md b/docs/learn/documentation/0.7.0/introduction/architecture.md
new file mode 100644
index 0000000..74470d1
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/introduction/architecture.md
@@ -0,0 +1,90 @@
+---
+layout: page
+title: Architecture
+---
+
+Samza is made up of three layers:
+
+1. A streaming layer.
+2. An execution layer.
+3. A processing layer.
+
+Samza provides out of the box support for all three layers.
+
+1. **Streaming:** [Kafka](http://kafka.apache.org/)
+2. **Execution:** [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html)
+3. **Processing:** [Samza API](../api/overview.html)
+
+These three pieces fit together to form Samza.
+
+![diagram-medium](/img/0.7.0/learn/documentation/introduction/samza-ecosystem.png)
+
+This architecture should be familiar to anyone that's used Hadoop.
+
+![diagram-medium](/img/0.7.0/learn/documentation/introduction/samza-hadoop.png)
+
+Before going in-depth on each of these three layers, it should be noted that Samza supports is not limited to these systems. Both Samza's execution and streaming layer are pluggable, and allow developers to implement alternatives if they prefer.
+
+### Kafka
+
+[Kafka](http://kafka.apache.org/) is a distributed pub/sub and message queueing system that provides at-least once messaging guarantees, and highly available partitions (i.e. a stream's partitions will be available, even if a machine goes down).
+
+In Kafka, each stream is called a "topic". Each topic is partitioned up, to make things scalable. When a "producer" sends a message to a topic, the producer provides a key, which is used to determine which partition the message should be sent to. Kafka "brokers", each of which are in charge of some partitions, receive the messages that the producer sends, and stores them on their disk in a log file. Kafka "consumers" can then read from a topic by getting messages from all of a topic's partitions.
+
+This has some interesting properties. First, all messages partitioned by the same key are guaranteed to be in the same Kafka topic partition. This means, if you wish to read all messages for a specific member ID, you only have to read the messages from the partition that the member ID is on, not the whole topic (assuming the topic is partitioned by member ID). Second, since a Kafka broker's file is a log, you can reference any point in the log file using an "offset". This offset determines where a consumer is in a topic/partition pair. After every message a consumer reads from a topic/partition pair, the offset is incremented.
+
+For more details on Kafka, see Kafka's [introduction](http://kafka.apache.org/introduction.html) and [design](http://kafka.apache.org/design.html) pages.
+
+### YARN
+
+[YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) (Yet Another Resource Negotiator) is Hadoop's next-generation cluster scheduler. It allows you to allocate a number of "containers" (processes) in a cluster of machines, and execute arbitrary commands on them.
+
+When an application interacts with YARN, it looks something like this:
+
+1. **Application**: I want to run command X on two machines with 512M memory
+2. **YARN**: Cool, where's your code?
+3. **Application**: http://path.to.host/jobs/download/my.tgz
+4. **YARN**: I'm running your job on node-1.grid and node-1.grid
+
+Samza uses YARN to manage:
+
+* Deployment
+* Fault tolerance
+* Logging
+* Isolation
+* Security
+* Locality
+
+This page covers a brief overview of YARN, but [this page](http://hortonworks.com/blog/apache-hadoop-yarn-background-and-an-overview/) from Hortonworks contains a much better overview.
+
+#### YARN Architecture
+
+YARN has three important pieces: a ResourceManager, a NodeManager, and an ApplicationMaster. In a YARN grid, every computer runs a NodeManager, which is responsible for running processes on the local machine. A ResourceManager talks to all of the NodeManagers to tell it what to run. Applications, in turn, talk to the ResourceManager when they wish to run something on the cluster. The flow, when starting a new application, goes from user application to YARN RM, to YARN NM. The third piece, the ApplicationMaster, is actually application-specific code that runs in the YARN cluster. It's responsible for managing the application's workload, asking for containers (usually, UNIX processes), and handling notifications when one of its containers fails.
+
+#### Samza and YARN
+
+Samza provides a YARN ApplicationMaster, and YARN job runner out of the box. The integration between Samza and YARN is outlined in the following diagram (different colors indicate different host machines).
+
+![diagram-small](/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png)
+
+The Samza client talks to the YARN RM when it wants to start a new Samza job. The YARN RM talks to a YARN NM to allocate space on the cluster for Samza's ApplicationMaster. Once the NM allocates space, it starts the Samza AM. After the Samza AM starts, it asks the YARN RM for one, or more, YARN containers to run Samza [TaskRunners](../container/task-runner.html). Again, the RM works with NMs to allocate space for the containers. Once the space has been allocated, the NMs start the Samza containers.
+
+### Samza
+
+Samza uses YARN and Kafka to provide a framework for stage-wise stream processing and partitioning. Everything, put together, looks like this (different colors indicate different host machines):
+
+![diagram-small](/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png)
+
+The Samza client uses YARN to run a Samza job. The Samza [TaskRunners](../container/task-runner.html) run in one, or more, YARN containers, and execute user-written Samza [StreamTasks](../api/overview.html). The input and output for the Samza StreamTasks come from Kafka brokers that are (usually) co-located on the same machines as the YARN NMs.
+
+### Example
+
+Let's take a look at a real example. Suppose that we wanted to count page views grouped by member ID. In SQL, it would look something like: SELECT COUNT(\*) FROM PageViewEvent GROUP BY member_id. Although Samza doesn't support SQL right now, the idea is the same. Two jobs are required to calculate this query: one to group messages by member ID, and the other to do the counting. The counting and grouping can't be done in the same Samza job because the input topic might not be partitioned by the member ID. Anyone familiar with Hadoop will recognize this as a Map/Reduce operation, where you first map data by a particular key, and then count in the reduce step.
+
+![diagram-large](/img/0.7.0/learn/documentation/introduction/group-by-example.png)
+
+The input topic is partitioned using Kafka. Each Samza process reads messages from one or more of the input topic's partitions, and emits them back out to a different Kafka topic keyed by the message's member ID attribute. The Kafka brokers receive these messages, and buffer them on disk until the second job (the counting job on the bottom of the diagram) reads the messages, and increments its counters.
+
+There are some neat things to consider about this example. First, we're leveraging the fact that Kafka topics are inherently partitioned. This lets us run one or more Samza processes, and assign them each some partitions to read from. Second, since we're guaranteed that, for a given key, all messages will be on the same partition, we can actually split up the aggregation (counting). For example, if the first job's output had four partitions, we could assign two partitions to the first count process, and the other two partitions to the second count process. We'd be guaranteed that for any give member ID, all of their messages will be consumed by either the first process or the second, but not both. This means we'll get accurate counts, even when partitioning. Third, the fact that we're using Kafka, which buffers messages on its brokers, also means that we don't have to worry as much about failures. If a process or machine fails, we can use YARN to start the process on another machine
 . When the process starts up again, it can get its last offset, and resume reading messages where it left off.
+
+## [Comparison Introduction &raquo;](../comparisons/introduction.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/introduction/background.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/introduction/background.md b/docs/learn/documentation/0.7.0/introduction/background.md
new file mode 100644
index 0000000..1437611
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/introduction/background.md
@@ -0,0 +1,55 @@
+---
+layout: page
+title: Background
+---
+
+This page provides some background about stream processing, describes what Samza is, and why it was built.
+
+### What is messaging?
+
+Messaging systems are a popular way of implementing near-realtime asynchronous computation. Messages can be added to a message queue (Active MQ, Rabbit MQ), pub-sub system (Kestrel, Kafka), or log aggregation system (Flume, Scribe) when something happens. Downstream "consumers" read messages from these systems, and process or take action based on the message contents.
+
+Suppose that you have a server that's serving web pages. You can have the web server send a "user viewed page" event to a messaging system. You might then have consumers:
+
+* Put the message into Hadoop
+* Count page views and update a dashboard
+* Trigger an alert if a page view fails
+* Send an email notification to another use
+* Join the page view event with the user's profile, and send the message back to the messaging system
+
+A messaging system lets you decouple all of this work from the actual web page serving.
+
+### What is stream processing?
+
+A messaging system is a fairly low-level piece of infrastructure---it stores messages and waits for consumers to consume them. When you start writing code that produces or consumes messages, you quickly find that there are a lot of tricky problems that have to be solved in the processing layer. Samza aims to help with these problems.
+
+Consider the counting example, above (count page views and update a dashboard). What happens when the machine that your consumer is running on fails, and your "current count" is lost. How do you recover? Where should the processor be run when it restarts? What if the underlying messaging system sends you the same message twice, or loses a message? Your counts will be off. What if you want to count page views grouped by the page URL? How can you do that in a distributed environment?
+
+Stream processing is a higher level of abstraction on top of messaging systems, and it's meant to address precisely this category of problems.
+
+### Samza
+
+Samza is a stream processing framework with the following features:
+
+* **Simpe API:** Samza provides a very simple call-back based "process message" API.
+* **Managed state:** Samza manages snapshotting and restoration of a stream processor's state. Samza will restore a stream processor's state to a snapshot consistent with the processor's last read messages when the processor is restarted. Samza is built to handle large amounts of state (even many gigabytes per partition).
+* **Fault tolerance:** Samza will work with YARN to transparently migrate your tasks whenever a machine in the cluster fails.
+* **Durability:** Samza uses Kafka to guarantee that no messages will ever be lost.
+* **Scalability:** Samza is partitioned and distributed at every level. Kafka provides ordered, partitioned, replayable, fault-tolerant streams. YARN provides a distributed environment for Samza containers to run in.
+* **Pluggable:** Though Samza works out of the box with Kafka and YARN, Samza provides a pluggable API that lets you run Samza with other messaging systems and execution environments.
+* **Processor isolation:** Samza works with Apache YARN, to give security and resource scheduling, and resource isolation through Linux CGroups.
+
+### Alternatives
+
+The open source stream processing systems that are available are actually quite young, and no single system offers a complete solution. Problems like how a stream processor's state should be managed, whether a stream should be buffered remotely on disk or not, what to do when duplicate messages are received or messages are lost, and how to model underlying messaging systems are all pretty new.
+
+Samza's main differentiators are:
+
+* Samza supports fault-tolerant local state. State can be thought of as tables that are split up and maintained with the processing tasks. State is itself modeled as a stream. When a processor is restarted, the state stream is entirely replayed to restore it.
+* Streams are ordered, partitioned, replayable, and fault tolerant.
+* YARN is used for processor isolation, security, and fault tolerance.
+* All streams are materialized to disk.
+
+For a more in-depth discussion on Samza, and how it relates to other stream processing systems, have a look at Samza's [Comparisons](../comparisons/introduction.html) documentation.
+
+## [Concepts &raquo;](concepts.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/introduction/concepts.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/introduction/concepts.md b/docs/learn/documentation/0.7.0/introduction/concepts.md
new file mode 100644
index 0000000..206133d
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/introduction/concepts.md
@@ -0,0 +1,58 @@
+---
+layout: page
+title: Concepts
+---
+
+This page gives an introduction to the high-level concepts in Samza.
+
+### Streams
+
+Samza processes *streams*. A stream is composed of immutable *messages* of a similar type or category. Example streams might include all the clicks on a website, or all the updates to a particular database table, or any other type of event data. Messages can be appended to a stream or read from a stream. A stream can have any number of readers and reading from a stream doesn't delete the message so a message written to a stream is effectively broadcast out to all readers. Messages can optionally have an associated key which is used for partitioning, which we'll talk about in a second.
+
+Samza supports pluggable *systems* that implement the stream abstraction: in Kafka a stream is a topic, in a database we might read a stream by consuming updates from a table, in Hadoop we might tail a directory of files in HDFS.
+
+![job](/img/0.7.0/learn/documentation/introduction/job.png)
+
+### Jobs
+
+A Samza *job* is code that performs a logical transformation on a set of input streams to append output messages to set of output streams.
+
+If scalability were not a concern streams and jobs would be all we would need. But to let us scale our jobs and streams we chop these two things up into smaller unit of parallelism below the stream and job, namely *partitions* and *tasks*.
+
+### Partitions
+
+Each stream is broken into one or more partitions. Each partition in the stream is a totally ordered sequence of messages.
+
+Each position in this sequence has a unique identifier called the *offset*. The offset can be a sequential integer, byte offset, or string depending on the underlying system implementation.
+
+Each message appended to a stream is appended to only one of the streams partitions. The assignment of the message to its partition is done with a key chosen by the writer (in the click example above, data might be partitioned by user id).
+
+![stream](/img/0.7.0/learn/documentation/introduction/stream.png)
+
+### Tasks
+
+A job is itself distributed by breaking it into multiple *tasks*. The *task* is the unit of parallelism of the job, just as the partition is to the stream. Each task consumes data from one partition for each of the job's input streams.
+
+The task processes messages from each of its input partitions *in order by offset*. There is no defined ordering between partitions.
+
+The position of the task in its input partitions can be represented by set of offsets, one for each partition.
+
+The number of tasks a job has is fixed and does not change (though the computational resources assigned to the job may go up and down). The number of tasks a job has also determines the maximum parallelism of the job as each task processes messages sequentially. There cannot be more tasks than input partitions (or there would be some task with no input).
+
+The partitions assigned to a task will never change: if a task is on a machine that fails the task will be restarted elsewhere still consuming the same stream partitions.
+
+![job-detail](/img/0.7.0/learn/documentation/introduction/job_detail.png)
+
+### Dataflow Graphs
+
+We can compose multiple jobs to create data flow graph where the nodes are streams containing data and the edges are jobs performing transformations. This composition is done purely through the streams the jobs take as input and output&mdash;the jobs are otherwise totally decoupled: They need not be implemented in the same code base, and adding, removing, or restarting a downstream job will not impact an upstream job.
+
+These graphs are often acyclic&mdash;that is, data usually doesn't flow from a job, through other jobs, back to itself. However this is not a requirement.
+
+![dag](/img/0.7.0/learn/documentation/introduction/dag.png)
+
+### Containers
+
+Partitions and tasks are both *logical* units of parallelism, they don't actually correspond to any particular assignment of computational resources (CPU, memory, disk space, etc). Containers are the unit of physical parallelism, and a container is essentially just a unix process (or linux [cgroup](http://en.wikipedia.org/wiki/Cgroups)). Each container runs one or more tasks. The number of tasks is determined automatically from the number of partitions in the input and is fixed, but the number of containers (and the cpu and memory resources associated with them) is specified by the user at run time and can be changed at any time.
+
+## [Architecture &raquo;](architecture.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/configuration-table.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/configuration-table.html b/docs/learn/documentation/0.7.0/jobs/configuration-table.html
new file mode 100644
index 0000000..41353b2
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/configuration-table.html
@@ -0,0 +1,224 @@
+<html>
+  <body>
+    <table cellspacing="2" border="1" cellpadding="2">
+      <tbody>
+        <tr><th>Name</th><th>Default</th><th>Description</th></tr>
+        <tr>
+          <td><strong>Job</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>job.factory.class</td>
+          <td>none</td>
+          <td><strong>Required:</strong> The job factory to use when running a task. This can be either samza.job.local.LocalJobFactory, or samza.job.yarn.YarnJobFactory.</td>
+        </tr>
+        <tr>
+          <td>job.name</td>
+          <td>none</td>
+          <td><strong>Required:</strong> The name of your job. This is the name that will appear on the Samza dashboard, when your job is running.</td>
+        </tr>
+        <tr>
+          <td>job.id</td>
+          <td>1</td>
+          <td>An ID string that is used to distinguish between multiple concurrent executions of the same Samza job.</td>
+        </tr>
+        <tr>
+          <td><strong>Task</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>task.class</td>
+          <td>none</td>
+          <td><strong>Required:</strong> The package name of the StreamTask to execute. For example, samza.task.example.MyStreamerTask.</td>
+        </tr>
+        <tr>
+          <td>task.execute</td>
+          <td>bin/run-task.sh</td>
+          <td>The command that a StreamJob should invoke to start the TaskRunner.</td>
+        </tr>
+        <tr>
+          <td>task.message.buffer.size</td>
+          <td>10000</td>
+          <td>The number of messages that the TaskRunner will buffer in the event loop queue, before it begins blocking StreamConsumers.</td>
+        </tr>
+        <tr>
+          <td>task.inputs</td>
+          <td>none</td>
+          <td><strong>Required:</strong> A CSV list of stream names that the TaskRunner should use to read messages from for your StreamTasks (e.g. page-view-event,service-metrics).</td>
+        </tr>
+        <tr>
+          <td>task.window.ms</td>
+          <td>-1</td>
+          <td>How often the TaskRunner should call window() on a WindowableTask. A negative number tells the TaskRunner to never call window().</td>
+        </tr>
+        <tr>
+          <td>task.commit.ms</td>
+          <td>60000</td>
+          <td>How often the TaskRunner should call writeCheckpoint for a partition.</td>
+        </tr>
+        <tr>
+          <td>task.command.class</td>
+          <td>samza.task.ShellCommandBuilder</td>
+          <td>The class to use to build environment variables for the task.execute command.</td>
+        </tr>
+        <tr>
+          <td>task.lifecycle.listeners</td>
+          <td>none</td>
+          <td>A CSV list of lifecycle listener names that the TaskRunner notify when lifecycle events occur (e.g. my-lifecycle-manager).</td>
+        </tr>
+        <tr>
+          <td>task.lifecycle.listener.%s.class</td>
+          <td>none</td>
+          <td>The class name for a lifecycle listener factory (e.g. task.lifecycle.listener.my-lifecycle-manager.class=foo.bar.MyLifecycleManagerFactory)</td>
+        </tr>
+        <tr>
+          <td>task.checkpoint.factory</td>
+          <td>none</td>
+          <td>The class name for the checkpoint manager to use (e.g. samza.task.state.KafkaCheckpointManagerFactory)</td>
+        </tr>
+        <tr>
+          <td>task.checkpoint.failure.retry.ms</td>
+          <td>10000</td>
+          <td>If readLastCheckpoint, or writeCheckpoint fails, the TaskRunner will wait this interval before retrying the checkpoint.</td>
+        </tr>
+        <tr>
+          <td colspan="1">task.opts</td>
+          <td colspan="1">none</td>
+          <td colspan="1">JVM options that should be attached to each JVM that is running StreamTasks. If you wish to reference the log directory from this parameter, use logs/. <span>If you wish to reference code in the Samza job's TGZ package use __package/.</span></td>
+        </tr>
+        <tr>
+          <td><strong>System</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>systems.%s.samza.consumer.factory</td>
+          <td>none</td>
+          <td>The StreamConsumerFactory class to use when creating a new StreamConsumer for this system (e.g. samza.stream.kafka.KafkaConsumerFactory).</td>
+        </tr>
+        <tr>
+          <td>systems.%s.samza.producer.factory</td>
+          <td>none</td>
+          <td>The StreamProducerFactory class to use when creating a new StreamProducer for this system (e.g. samza.stream.kafka.KafkaProducerFactory).</td>
+        </tr>
+        <tr>
+          <td>systems.%s.samza.partition.manager</td>
+          <td>none</td>
+          <td>The PartitionManager class to use when fetching partition information about streams for the system (e.g. samza.stream.kafka.KafkaPartitionManager).</td>
+        </tr>
+        <tr>
+          <td>systems.%s.producer.reconnect.interval.ms</td>
+          <td>10000</td>
+          <td>If a producer fails, the TaskRunner will wait this interval before retrying.</td>
+        </tr>
+        <tr>
+          <td>systems.%s.*</td>
+          <td>none</td>
+          <td>For both Kafka and Databus, any configuration you supply under this namespace will be given to the underlying Kafka consumer/producer, and Databus consumer/producer. This is useful for configuring things like autooffset.reset, socket buffer size, fetch size, batch size, etc.</td>
+        </tr>
+        <tr>
+          <td><strong>Stream</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>streams.%s.system</td>
+          <td>none</td>
+          <td>The name of the system associated with this stream (e.g. kafka-aggregate-tracking). This name must match with a system defined in the configuration file.</td>
+        </tr>
+        <tr>
+          <td>streams.%s.stream</td>
+          <td>none</td>
+          <td>The name of the stream in the system (e.g. PageViewEvent).</td>
+        </tr>
+        <tr>
+          <td>streams.%s.serde</td>
+          <td>none</td>
+          <td>The serde to use to serialize and deserialize messages for this stream. If undefined, the TaskRunner will try to fall back to the default serde, if it's defined.</td>
+        </tr>
+        <tr>
+          <td>streams.%s.consumer.reset.offset</td>
+          <td>false</td>
+          <td>If set to true, the TaskRunner will ignore the last checkpoint offset for this stream, and use null as the offset for the stream instead. In the case of Kafka's consumer, it will fall back to autooffset.reset. In the case of Databus' consumer, it will fall back to SCN 0.</td>
+        </tr>
+        <tr>
+          <td>streams.%s.consumer.failure.retry.ms</td>
+          <td>10000</td>
+          <td>If a StreamConsumer fails, the TaskRunner will wait this interval before retrying.</td>
+        </tr>
+        <tr>
+          <td>streams.%s.consumer.max.bytes.per.sec</td>
+          <td>none</td>
+          <td>The maximum number of bytes that the TaskRunner will allow from all partitions that it's reading for this stream. For example, if you have an input stream with two partitions, and 1 MB/sec max, then the maximum bytes the TaskRunner will read per second from all of the input stream's partitions is 1 MB/sec.</td>
+        </tr>
+        <tr>
+          <td>streams.%s.producer.reconnect.interval.ms</td>
+          <td>10000</td>
+          <td>If a StreamProducer fails, the TaskRunner will wait this interval before retrying.</td>
+        </tr>
+        <tr>
+          <td><strong>Serdes</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>serializers.registry.%s.class</td>
+          <td>none</td>
+          <td>The name of a class that implements both SerializerFactory and DeserializerFactory (e.g. serializers.registry.json.class=samza.serializers.JsonSerdeFactory).</td>
+        </tr>
+        <tr>
+          <td>serializers.default</td>
+          <td>none</td>
+          <td>The default serde to use, if one is not defined for an input or output stream (e.g. serializers.default=json).</td>
+        </tr>
+        <tr>
+          <td><strong>YARN</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>yarn.package.path</td>
+          <td>none</td>
+          <td>The tgz location of your Samza job. This tgz file is well structured. See the YARN section for details.</td>
+        </tr>
+        <tr>
+          <td>yarn.container.memory.mb</td>
+          <td>768</td>
+          <td>How much memory to ask for (per-container), when Samza is starting a YARN container.</td>
+        </tr>
+        <tr>
+          <td>yarn.container.count</td>
+          <td>1</td>
+          <td>How many containers to start when a Samza job is started in YARN. Partitions are divided evenly among the containers.</td>
+        </tr>
+        <tr>
+          <td colspan="1">yarn.am.opts</td>
+          <td colspan="1">none</td>
+          <td colspan="1"><span>JVM options that should be attached to each JVM that is running the ApplicationMaster. If you wish to reference the log directory from this parameter, use logs/. If you wish to reference code in the Samza job's TGZ package use __package/.</span></td>
+        </tr>
+        <tr>
+          <td><strong>Metrics</strong></td>
+          <td> </td>
+          <td> </td>
+        </tr>
+        <tr>
+          <td>metrics.reporter.%s.class</td>
+          <td>none</td>
+          <td>The package and class for a metrics reporter (e.g. metrics.reporter.foo-bar.class=samza.metrics.reporter.MetricsSnapshotReporter).</td>
+        </tr>
+        <tr>
+          <td>metrics.reporter.%s.window.ms</td>
+          <td>10000</td>
+          <td>How often the TaskRunner tells the metrics reporter to send update or send its metrics.</td>
+        </tr>
+        <tr>
+          <td>metrics.reporters</td>
+          <td>none</td>
+          <td>A CSV list of metric reporter names (e.g. metrics.reporters=foo-bar).</td>
+        </tr>
+      </tbody>
+    </table>
+  </body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/configuration.md b/docs/learn/documentation/0.7.0/jobs/configuration.md
new file mode 100644
index 0000000..01035ba
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/configuration.md
@@ -0,0 +1,45 @@
+---
+layout: page
+title: Configuration
+---
+
+All Samza jobs have a configuration file that defines the job. A very basic configuration file looks like this:
+
+```
+# Job
+job.factory.class=samza.job.local.LocalJobFactory
+job.name=hello-world
+
+# Task
+task.class=samza.task.example.MyJavaStreamerTask
+task.inputs=example-stream
+
+# Serializers
+serializers.registry.json.class=samza.serializers.JsonSerdeFactory
+serializers.default=json
+
+# Streams
+streams.example-stream.system=example-system
+streams.example-stream.stream=some-stream
+
+# Systems
+systems.example-system.samza.consumer.factory=samza.stream.example.ExampleConsumerFactory
+systems.example-system.samza.partition.manager=samza.stream.example.ExamplePartitionManager
+```
+
+There are five major sections to a configuration file. The job section defines things like the name of the job, and whether to use the YarnJobFactory or LocalJobFactory. The task section is where you specify the class name for your StreamTask. It's also where you define what the input streams are for your task. The system section defines systems that you can read from. Usually, you'll define a Kafka system, if you're reading from Kafka. After that you'll need to define the stream(s) that you want to read from, which systems they're coming from, and how to deserialize objects from the stream.
+
+### Required Configuration
+
+Configuration keys that absolutely must be defined for a Samza job are:
+
+* job.factory.class
+* job.name
+* task.class
+* task.inputs
+
+### Configuration Keys
+
+A complete list of configuration keys can be found on the [Configuration Table](configuration-table.html) page.
+
+## [Packaging &raquo;](packaging.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/job-runner.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/job-runner.md b/docs/learn/documentation/0.7.0/jobs/job-runner.md
new file mode 100644
index 0000000..4c2ab4c
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/job-runner.md
@@ -0,0 +1,46 @@
+---
+layout: page
+title: JobRunner
+---
+
+Samza jobs are started using a script called run-job.sh.
+
+```
+samza-example/target/bin/run-job.sh \
+  --config-factory=samza.config.factories.PropertiesConfigFactory \
+  --config-path=file://$PWD/config/hello-world.properties
+```
+
+You provide two parameters to the run-job.sh script. One is the config location, and the other is a factory class that is used to read your configuration file. The run-job.sh script is actually executing a Samza class called JobRunner. The JobRunner uses your ConfigFactory to get a Config object from the config path.
+
+```
+public interface ConfigFactory {
+  Config getConfig(URI configUri);
+}
+```
+
+The Config object is just a wrapper around Map<String, String>, with some nice helper methods. Out of the box, Samza ships with the PropertiesConfigFactory, but developers can implement any kind of ConfigFactory they wish.
+
+Once the JobRunner gets your configuration, it gives your configuration to the StreamJobFactory class defined by the "job.factory" property. Samza ships with two job factory implementations: LocalJobFactory and YarnJobFactory. The StreamJobFactory's responsibility is to give the JobRunner a job that it can run.
+
+```
+public interface StreamJob {
+  StreamJob submit();
+
+  StreamJob kill();
+
+  ApplicationStatus waitForFinish(long timeoutMs);
+
+  ApplicationStatus waitForStatus(ApplicationStatus status, long timeoutMs);
+
+  ApplicationStatus getStatus();
+}
+```
+
+Once the JobRunner gets a job, it calls submit() on the job. This method is what tells the StreamJob implementation to start the TaskRunner. In the case of LocalJobRunner, it uses a run-task.sh script to execute the TaskRunner in a separate process, which will start one TaskRunner locally on the machine that you ran run-job.sh on.
+
+![diagram](/img/0.7.0/learn/documentation/container/job-flow.png)
+
+This flow differs slightly when you use YARN, but we'll get to that later.
+
+## [Configuration &raquo;](configuration.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/logging.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/logging.md b/docs/learn/documentation/0.7.0/jobs/logging.md
new file mode 100644
index 0000000..9ef9ca1
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/logging.md
@@ -0,0 +1,53 @@
+---
+layout: page
+title: Logging
+---
+
+Samza uses [SLF4J](http://www.slf4j.org/) for all of its logging. By default, only slf4j-api is used, so you must add an SLF4J runtime dependency to your Samza packages for whichever underlying logging platform you wish to use.
+
+### Log4j
+
+The [hello-samza](/startup/hello-samza/0.7.0) project shows how to use [log4j](http://logging.apache.org/log4j/1.2/) with Samza. To turn on log4j logging, you just need to make sure slf4j-log4j12 is in your Samza TaskRunner's classpath. In Maven, this can be done by adding the following dependency to your Samza package project.
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>runtime</scope>
+      <version>1.6.2</version>
+    </dependency>
+
+If you're not using Maven, just make sure that slf4j-log4j12 ends up in your Samza package's lib directory.
+
+#### log4j.xml
+
+Samza's [run-class.sh](packaging.html) script will automatically set the following setting if log4j.xml exists in your [Samza package's](packaging.html) lib directory.
+
+    -Dlog4j.configuration=file:$base_dir/lib/log4j.xml
+
+<!-- TODO add notes showing how to use task.opts for gc logging
+#### task.opts
+-->
+
+### Log Directory
+
+Samza will look for the _SAMZA_\__LOG_\__DIR_ environment variable when it executes. If this variable is defined, all logs will be written to this directory. If the environment variable is empty, or not defined, then Samza will use /tmp. This environment variable can also be referenced inside log4j.xml files.
+
+### Garbage Collection Logging
+
+Samza's will automatically set the following garbage collection logging setting, and will output it to _$SAMZA_\__LOG_\__DIR_/gc.log.
+
+    -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log
+
+#### Rotation
+
+In older versions of Java, it is impossible to have GC logs roll over based on time or size without the use of a secondary tool. This means that your GC logs will never be deleted until a Samza job ceases to run. As of [Java 6 Update 34](http://www.oracle.com/technetwork/java/javase/2col/6u34-bugfixes-1733379.html), and [Java 7 Update 2](http://www.oracle.com/technetwork/java/javase/7u2-relnotes-1394228.html), [new GC command line switches](http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6941923) have been added to support this functionality. If you are using a version of Java that supports GC log rotation, it's highly recommended that you turn it on.
+
+### YARN
+
+When a Samza job executes on a YARN grid, the _$SAMZA_\__LOG_\__DIR_ environment variable will point to a directory that is secured such that only the user executing the Samza job can read and write to it, if YARN is [securely configured](http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/ClusterSetup.html).
+
+#### STDOUT
+
+YARN pipes all STDOUT and STDERR output to logs/stdout and logs/stderr, respectively. These files are never rotated.
+
+## [Application Master &raquo;](../yarn/application-master.html)


[12/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/stream.graffle
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/stream.graffle b/docs/img/0.7.0/learn/documentation/introduction/stream.graffle
new file mode 100644
index 0000000..5281bf5
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/introduction/stream.graffle
@@ -0,0 +1,2670 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN" "http://www.apple.com/DTDs/PropertyList-1.0.dtd">
+<plist version="1.0">
+<dict>
+	<key>ActiveLayerIndex</key>
+	<integer>0</integer>
+	<key>ApplicationVersion</key>
+	<array>
+		<string>com.omnigroup.OmniGrafflePro.MacAppStore</string>
+		<string>139.18</string>
+	</array>
+	<key>AutoAdjust</key>
+	<true/>
+	<key>BackgroundGraphic</key>
+	<dict>
+		<key>Bounds</key>
+		<string>{{0, 0}, {576.00002479553223, 733}}</string>
+		<key>Class</key>
+		<string>SolidGraphic</string>
+		<key>ID</key>
+		<integer>2</integer>
+		<key>Style</key>
+		<dict>
+			<key>shadow</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+			<key>stroke</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+		</dict>
+	</dict>
+	<key>BaseZoom</key>
+	<integer>0</integer>
+	<key>CanvasOrigin</key>
+	<string>{0, 0}</string>
+	<key>ColumnAlign</key>
+	<integer>1</integer>
+	<key>ColumnSpacing</key>
+	<real>36</real>
+	<key>CreationDate</key>
+	<string>2013-07-28 22:22:56 +0000</string>
+	<key>Creator</key>
+	<string>Jay Kreps</string>
+	<key>DisplayScale</key>
+	<string>1 0/72 in = 1 0/72 in</string>
+	<key>GraphDocumentVersion</key>
+	<integer>8</integer>
+	<key>GraphicsList</key>
+	<array>
+		<dict>
+			<key>Bounds</key>
+			<string>{{61, 10}, {138, 18}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>99</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs30 \cf0 A Partitioned Stream}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{241.5, 93.238006591796875}, {57, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>98</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 next append}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>97</integer>
+			<key>Points</key>
+			<array>
+				<string>{236, 99}</string>
+				<string>{199, 143.73800659179688}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+					<key>Width</key>
+					<real>0.5</real>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>96</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>96</integer>
+			<key>Points</key>
+			<array>
+				<string>{236, 99}</string>
+				<string>{186, 99}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+					<key>Width</key>
+					<real>0.5</real>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>95</integer>
+			<key>Points</key>
+			<array>
+				<string>{236, 99}</string>
+				<string>{212, 55}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+					<key>Width</key>
+					<real>0.5</real>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>96</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{12, 138.72201098632814}, {50, 13}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>11</real>
+			</dict>
+			<key>ID</key>
+			<integer>92</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 partition 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{179, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>90</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 8}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>89</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 7}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>88</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 6}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>87</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 5}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>86</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 4}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>85</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 3}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>84</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>83</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 152.9680093688965}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>82</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{179, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>80</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Pattern</key>
+					<integer>1</integer>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>79</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>78</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>77</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>76</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>75</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>74</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>73</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 132.47600936889648}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>72</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{12, 93.238006011962895}, {50, 13}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>11</real>
+			</dict>
+			<key>ID</key>
+			<integer>71</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 partition 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>68</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 7}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>67</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 6}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>66</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 5}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>65</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 4}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>64</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 3}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>63</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>62</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 106.98400439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>61</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>58</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Pattern</key>
+					<integer>1</integer>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>57</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>56</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>55</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>54</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>53</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>52</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 86.49200439453125}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>51</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{12, 47.254001617431641}, {50, 13}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>11</real>
+			</dict>
+			<key>ID</key>
+			<integer>50</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 partition 0}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{193, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>44</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 9}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{179, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>43</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 8}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>42</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 7}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>41</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 6}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>40</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 5}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>39</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 4}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>38</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 3}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>37</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>36</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 61}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>35</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Width</key>
+					<real>0.0</real>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{193, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>29</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Pattern</key>
+					<integer>1</integer>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{179, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>28</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{165, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>27</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{151, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>26</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>25</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{123, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>24</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{109, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>23</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{95, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>22</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{81, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>21</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{67, 40.507999999999996}, {14, 25.492000579833984}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>19</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+	</array>
+	<key>GridInfo</key>
+	<dict/>
+	<key>GuidesLocked</key>
+	<string>NO</string>
+	<key>GuidesVisible</key>
+	<string>YES</string>
+	<key>HPages</key>
+	<integer>1</integer>
+	<key>ImageCounter</key>
+	<integer>1</integer>
+	<key>KeepToScale</key>
+	<false/>
+	<key>Layers</key>
+	<array>
+		<dict>
+			<key>Lock</key>
+			<string>NO</string>
+			<key>Name</key>
+			<string>Layer 1</string>
+			<key>Print</key>
+			<string>YES</string>
+			<key>View</key>
+			<string>YES</string>
+		</dict>
+	</array>
+	<key>LayoutInfo</key>
+	<dict>
+		<key>Animate</key>
+		<string>NO</string>
+		<key>circoMinDist</key>
+		<real>18</real>
+		<key>circoSeparation</key>
+		<real>0.0</real>
+		<key>layoutEngine</key>
+		<string>dot</string>
+		<key>neatoSeparation</key>
+		<real>0.0</real>
+		<key>twopiSeparation</key>
+		<real>0.0</real>
+	</dict>
+	<key>LinksVisible</key>
+	<string>NO</string>
+	<key>MagnetsVisible</key>
+	<string>NO</string>
+	<key>MasterSheets</key>
+	<array/>
+	<key>ModificationDate</key>
+	<string>2013-07-28 22:33:38 +0000</string>
+	<key>Modifier</key>
+	<string>Jay Kreps</string>
+	<key>NotesVisible</key>
+	<string>NO</string>
+	<key>Orientation</key>
+	<integer>2</integer>
+	<key>OriginVisible</key>
+	<string>NO</string>
+	<key>PageBreaks</key>
+	<string>YES</string>
+	<key>PrintInfo</key>
+	<dict>
+		<key>NSBottomMargin</key>
+		<array>
+			<string>float</string>
+			<string>41</string>
+		</array>
+		<key>NSHorizonalPagination</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAhOU051bWJlcgCEhAdOU1ZhbHVlAISECE5TT2JqZWN0AIWEASqEhAFxlwCG</string>
+		</array>
+		<key>NSLeftMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSPaperSize</key>
+		<array>
+			<string>size</string>
+			<string>{612.00002479553223, 792}</string>
+		</array>
+		<key>NSPrintReverseOrientation</key>
+		<array>
+			<string>int</string>
+			<string>0</string>
+		</array>
+		<key>NSRightMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSTopMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+	</dict>
+	<key>PrintOnePage</key>
+	<false/>
+	<key>ReadOnly</key>
+	<string>NO</string>
+	<key>RowAlign</key>
+	<integer>1</integer>
+	<key>RowSpacing</key>
+	<real>36</real>
+	<key>SheetTitle</key>
+	<string>Canvas 1</string>
+	<key>SmartAlignmentGuidesActive</key>
+	<string>YES</string>
+	<key>SmartDistanceGuidesActive</key>
+	<string>YES</string>
+	<key>UniqueID</key>
+	<integer>1</integer>
+	<key>UseEntirePage</key>
+	<false/>
+	<key>VPages</key>
+	<integer>1</integer>
+	<key>WindowInfo</key>
+	<dict>
+		<key>CurrentSheet</key>
+		<integer>0</integer>
+		<key>ExpandedCanvases</key>
+		<array>
+			<dict>
+				<key>name</key>
+				<string>Canvas 1</string>
+			</dict>
+		</array>
+		<key>Frame</key>
+		<string>{{575, -38}, {711, 872}}</string>
+		<key>ListView</key>
+		<true/>
+		<key>OutlineWidth</key>
+		<integer>142</integer>
+		<key>RightSidebar</key>
+		<false/>
+		<key>ShowRuler</key>
+		<true/>
+		<key>Sidebar</key>
+		<true/>
+		<key>SidebarWidth</key>
+		<integer>120</integer>
+		<key>VisibleRegion</key>
+		<string>{{0, 0}, {576, 733}}</string>
+		<key>Zoom</key>
+		<real>1</real>
+		<key>ZoomValues</key>
+		<array>
+			<array>
+				<string>Canvas 1</string>
+				<real>1</real>
+				<real>1</real>
+			</array>
+		</array>
+	</dict>
+</dict>
+</plist>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/stream.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/stream.png b/docs/img/0.7.0/learn/documentation/introduction/stream.png
new file mode 100644
index 0000000..e190041
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/stream.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-1.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-1.png b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-1.png
new file mode 100644
index 0000000..bf2155b
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-1.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-2.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-2.png b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-2.png
new file mode 100644
index 0000000..22f7eeb
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-2.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-3.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-3.png b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-3.png
new file mode 100644
index 0000000..3c9ff05
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/motivation/data-processing-spectrum-3.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png b/docs/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png
new file mode 100644
index 0000000..949a2f0
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/apache-egg-logo.png
----------------------------------------------------------------------
diff --git a/docs/img/apache-egg-logo.png b/docs/img/apache-egg-logo.png
new file mode 100644
index 0000000..c04e70d
Binary files /dev/null and b/docs/img/apache-egg-logo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/samza-icon.png
----------------------------------------------------------------------
diff --git a/docs/img/samza-icon.png b/docs/img/samza-icon.png
new file mode 100644
index 0000000..4aa6c79
Binary files /dev/null and b/docs/img/samza-icon.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
new file mode 100644
index 0000000..9f9c80f
--- /dev/null
+++ b/docs/index.md
@@ -0,0 +1,34 @@
+---
+layout: default
+---
+
+## What is Samza?
+
+<!-- TODO remove samza code warning when we get our git repo setup -->
+
+*NOTE: The Samza code is not available until we get an Apache git repository. Follow along [here](https://issues.apache.org/jira/browse/INFRA-6617).*
+
+Apache Samza is a distributed stream processing framework. It uses <a target="_blank" href="http://kafka.apache.org">Apache Kafka</a> for messaging, and <a target="_blank" href="http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html">Apache Hadoop YARN</a> to provide fault tolerance, processor isolation, security, and resource management.
+
+* **Simpe API:** Unlike most low-level messaging system APIs, Samza provides a very simple call-back based "process message" API that should be familiar to anyone that's used Map/Reduce.
+* **Managed state:** Samza manages snapshotting and restoration of a stream processor's state. Samza will restore a stream processor's state to a snapshot consistent with the processor's last read messages when the processor is restarted.
+* **Fault tolerance:** Samza will work with YARN to restart your stream processor if there is a machine or processor failure.
+* **Durability:** Samza uses Kafka to guarantee that messages will be processed in the order they were written to a partition, and that no messages will ever be lost.
+* **Scalability:** Samza is partitioned and distributed at every level. Kafka provides ordered, partitioned, re-playable, fault-tolerant streams. YARN provides a distributed environment for Samza containers to run in.
+* **Pluggable:** Though Samza works out of the box with Kafka and YARN, Samza provides a pluggable API that lets you run Samza with other messaging systems and execution environments.
+* **Processor isolation:** Samza works with Apache YARN, which supports processor security through Hadoop's security model, and resource isolation through Linux CGroups.
+
+Check out [Hello Samza](/startup/hello-samza/0.7.0) to try Samza. Read the [Background](/learn/documentation/0.7.0/introduction/background.html) page to learn more about Samza.
+
+### Limitations
+
+* We are just moving our code to open source. This newly open sourced version has a few limitations:
+** It depends on a snapshot version of Kafka that will not officially be released for a few months
+** This branch represents our trunk, not the production version at LinkedIn. This rollout is pending.
+** We have not yet fully implemented our plans around fault-tolerance semantics.
+
+### Pardon our Dust
+
+Apache Samza is currently undergoing incubation at the [Apache Software Foundation](http://www.apache.org/).
+
+![Apache Incubator Logo](img/apache-egg-logo.png)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/api/overview.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/api/overview.md b/docs/learn/documentation/0.7.0/api/overview.md
new file mode 100644
index 0000000..fb5f479
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/api/overview.md
@@ -0,0 +1,99 @@
+---
+layout: page
+title: API Overview
+---
+
+When writing a stream processor for Samza, you must implement the StreamTask interface:
+
+```
+/** User processing tasks implement this. */
+public interface StreamTask {
+  void process(MessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator);
+}
+```
+
+When Samza runs your task, the process method will be called once for each message that Samza receives from your task's input streams. The envelope contains three things of importance: the message, the key), and the stream that the message came from:
+
+```
+/** This class is given to a StreamTask once for each message that it receives. */
+public interface MessageEnvelope {
+  /** A deserialized message. */
+  <M> M getMessage();
+  
+  /** A deserialized key. */
+  <K> K getKey();
+  
+  /** The stream that this message came from. */
+  Stream getStream();
+}
+```
+
+Notice that the getStream() method returns a Stream object, not a String, as you might expect. This is because a Samza Stream actually consists of a name, a system, and a stream. The name is what you call the stream in your Samza configuration file. The system is the name of the cluster that the stream came from (e.g. kafka-aggreate-tracking, databus, etc). The system name is also defined in your Samza configuration file. Lastly, the actual stream is available. For Kafka, this would be the Kafka topic's name.
+
+```
+/** A name/system/stream tuple that represents a Samza stream. */
+public class Stream {
+  /** The name of the stream, if the stream is defined in a Samza job's
+      configuration. If not, this is null. */
+  public String getName() { ... }
+
+  /** The system name that this stream is associated with. This is also
+      defined in a Samza job's configuration. */
+  public String getSystem() { ... }
+
+  /** The stream name for the system. */
+  public String getStream() { ... }
+}
+```
+
+To make this a bit clearer, let me show you an example. A Samza job's configuration might have:
+
+```
+# the stream
+streams.page-view-event.stream=PageViewEvent
+streams.page-view-event.system=kafka
+streams.page-view-event.serde=json
+
+# the system
+systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager
+systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory
+systems.kafka.samza.producer.factory=samza.stream.kafka.KafkaProducerFactory
+...
+```
+
+I this example, getName would return page-view-event, getSystem would return kafka, and getStream would return PageViewEvent. If you've got more than one input stream feeding into your StreamTask, you can use the getStream() object to determine what kind of message you've received.
+
+What about sending messages? If you take a look at the process() method in StreamTask, you'll see that you get a MessageCollector.
+
+```
+/** When a task wishes to send a message, it uses this class. */
+public interface MessageCollector {
+  void send(KeyedMessageEnvelope envelope);
+}
+```
+
+<!-- TODO I think we're getting rid of KeyedMessageEnvelope in Jay's API change for state management. -->
+
+The collector takes KeyedMessageEnvelope, which extends the normal MessageEnvelope to allow tasks to supply a partition key when sending the message. The partition key, if supplied, is used to determine which partition of a stream a message is destined for.
+
+```
+/** A message envelope that has a key. */
+public interface KeyedMessageEnvelope extends MessageEnvelope {
+  <K> K getKey();
+}
+```
+
+And, putting it all together:
+
+```
+class MyStreamerTask extends StreamTask {
+  def process(envelope: MessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
+    val msg = envelope.getMessage[GenericRecord]
+    collector.send(new OutgoingMessageEnvelope(new Stream("kafka", "SomeTopicPartitionedByMemberId"), msg.get("member_id"), msg))
+  }
+}
+```
+
+This is a simplistic example that just reads from a stream, and sends the messages to SomeTopicPartitionedByMemberId, partitioned by the message's member ID.
+
+## [TaskRunner &raquo;](../container/task-runner.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/comparisons/introduction.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/comparisons/introduction.md b/docs/learn/documentation/0.7.0/comparisons/introduction.md
new file mode 100644
index 0000000..bb42d9b
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/comparisons/introduction.md
@@ -0,0 +1,50 @@
+---
+layout: page
+title: Comparison Introduction
+---
+
+Here are a few of the high-level design decisions that we think make Samza a bit different from what we have seen elsewhere.
+
+### The Stream Model
+
+Streams are the input and output to Samza jobs. Samza has a very strong model of a stream&mdash;they are more then just a simple message exchange mechanism. A stream in Samza is a partitioned, ordered, replayable, multi-subscriber, lossless sequence of messages. The stream acts as a buffer that isolates processing stages from one another.
+
+This stronger model requires persistence, fault-tolerance, and buffering in the stream implementation, but it has several benefits.
+
+First, delays in a downstream processing stage cannot block an upstream stage. A Samza job can stop and do a few minutes (or even hours) of blocking processing on accumulated data without causing any backlog for upstream jobs. Our goal is to be able to model all the asynchronous data processing a large company might do as a single unified dataflow graph, while still isolating these jobs which may be written, owned,  and run by different people in different code bases with varying SLAs.
+
+This is motivated in part by our experience building analogous (offline) processing pipelines in Hadoop. In Hadoop the output between processing stages are files and the processing stages MapReduce jobs. We have found that this strong isolation between stages makes it possible to have literally hundreds of loosely coupled jobs that comprise an offline processing ecosystem. Our goal is to be able to replicate this kind of rich ecosystem in the near-real-time setting.
+
+The second benefit of this stronger model is that all stages are multi-subscriber. In practical terms this means that if one person adds a set of processing flows that create output data streams, others can see it, consume it, and build on it, without any central repository or release schedule that ties these things together. As a happy side-effect this makes debugging flows very easy as you can effectively attach to the output of any stage and "tail" its output.
+
+Finally this strong stream model makes each processing stage completely independent which in turns greatly simplifies the implementation of many of the frameworks features. Each stage need only be concerned with its own inputs and outputs, there is no need to handle replaying large subgraphs which would require central control over otherwise independent jobs.
+
+The tradeoff is that this stronger stream model requires durability and persistence. We were willing to make this tradeoff for two reasons. First we think that MapReduce and HDFS have shown that durability can be done at high-throughput while offering near-limitless disk space. This lead us to develop Kafka which allows hundreds of MB/sec of replicated throughput and many TBs of disk space per node. We think this largely neutralizes the downside of persistence. MapReduce has occationally been criticized for its persistence-happy approach to processing. However this criticism is particularly inapplicable to stream processing. Batch processing like MapReduce often is used for processing large historical windows in a very short period of time (i.e. query a month of data in ten minutes); stream processing, on the other hand, mostly needs only keep up with the steady-state flow of data (i.e. process 10 minutes worth of data in 10 minutes). This means that the raw throughput requirements 
 for stream processing are, generally, several orders of magnitude lower than batch processes.
+
+### State
+
+We have put particular effort into allowing Samza jobs to manage large amounts of partitioned local state by providing out-of-the-box support for key-value access to a large local dataset.
+
+This means that you can view a Samza job as being both a piece of processing code, but also a co-partitioned "table" of state. This allows rich local queries and scans against this state. These tables are made fault-tolerant by producing a "changelog" stream which is used to restore the state of the table on fail-over. This stream is just another Samza stream, it can even be used as input for other jobs.
+
+![Stateful Processing](/img/samza_state.png)
+
+In our experience most processing flows require joins against other data sourceIn the absence of state maintenance, any joining or aggregation has to be done by querying an external data system. This tends to be one or two orders of magnitude slower than sequential processing. For example per-node throughput for Kafka would easily be in the 100k-500k messages/sec range (depending on message size) but remote queries against a key-value store tend to be closer to 1-5k queries-per-second per node.
+
+Worse mixing in queries from throughput-oriented stream processing on databases and services that also support live user traffic with low latency is a recipe for disaster. By offloading this into the stream processing system you effectively isolate the high-throughput stream processing from low-latency systems.
+
+By instead moving the data to the processing remote communication is completely eliminated for reads. If the data, once partitioned, fits in memory, then these lookups will be purely in memory and can run at outrageously fast rates.
+
+This pattern is not always appropriate and not required (nothing prevents external calls). To make use of this approach you must be able to produce a feed of changes from your databases, which not everyone can do. It also may be the case that much of the logic required to access the data properly is in an online service. In this case calling the service from your Samza job may be more convenient.
+
+### Execution Framework
+
+One final decision we made was to not build a custom distributed execution system in Samza. Instead execution is pluggable and currently completely handled by YARN. This has two benefits.
+
+The first benefit is practical&mdash;there is another team of smart people working on the execution framework. YARN supports a rich set of features around resource quotas and security and is developing at a rapid pace. This allows you to control both what portion of the cluster is allocated to which users and groups of users and also control the resource utilization on individual nodes (CPU, memory, etc) via CGroups. YARN is run at massive scale to support Hadoop and will likely become an ubiquitous layer. Since Samza runs entirely through YARN there is no separate daemons or masters to run beyond the YARN cluster itself (in other words if you already have Kafka and YARN you don't need to install anything to run Samza jobs).
+
+Secondly our integration with YARN is completely componentized. It exists in a separate package with no build time dependency on the main framework. This allows replacing YARN with other virtualization frameworks. In particular we are interested in adding direct AWS integration. Many companies run in AWS which is itself a virtualization framework, which, for Samza's purposes is equivalent to YARN--it allows you to create and destroy virtual "container" machines and guarantees fixed resources for these containers. Since stream processing jobs run "forever" it is a bit silly to run a YARN cluster inside AWS and then try to fill up this cluster with individual jobs. Instead a more sensible approach would just to directly allocate a set of EC2 instances for your jobs.
+
+We think there will be a lot of innovation both in open source virtualization frameworks like Mesos and YARN and in commercial cloud providers like Amazon so integrating with these makes sense.
+
+## [MUPD8 &raquo;](mupd8.html)


[03/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/resources/scalate/css/bootstrap.min.css
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/resources/scalate/css/bootstrap.min.css b/samza-yarn/src/main/resources/scalate/css/bootstrap.min.css
new file mode 100644
index 0000000..c10c7f4
--- /dev/null
+++ b/samza-yarn/src/main/resources/scalate/css/bootstrap.min.css
@@ -0,0 +1,9 @@
+/*!
+ * Bootstrap v2.3.1
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world @twitter by @mdo and @fat.
+ */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{width:auto\9;height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img,.google-maps img{max-widt
 h:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,html input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}@media print{*{color:#000!important;text-shadow:none!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) 
 ")"}.ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}@page{margin:.5cm}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover,a:focus{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-hei
 ght:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127
 659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*wid
 th:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-flui
 d .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-ch
 ild{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}a.muted:hover,a.muted:focus{color:#808080}.text-warning{color:#c09853}a.t
 ext-warning:hover,a.text-warning:focus{color:#a47e3c}.text-error{color:#b94a48}a.text-error:hover,a.text-error:focus{color:#953b39}.text-info{color:#3a87ad}a.text-info:hover,a.text-info:focus{color:#2d6987}.text-success{color:#468847}a.text-success:hover,a.text-success:focus{color:#356635}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1,h2,h3{line-height:40px}h1{font-size:38.5px}h2{font-size:31.5px}h3{font-size:24.5px}h4{font-size:17.5px}h5{font-size:14px}h6{font-size:11.9px}h1 small{font-size:24.5px}h2 small{font-size:17.5px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{mar
 gin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}ul.inline,ol.inline{margin-left:0;list-style:none}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;padding-right:5px;padding-left:5px;*zoom:1}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal{*zoom:1}.dl-horizontal:before,.dl-horizontal:after{display:table;line-height:0;content:""}.dl-horizontal:after{clear:both}.dl-horizontal dt{float:left;width:160px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25}b
 lockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;white-space:nowrap;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px
  solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type=
 "url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555;vertical-align:middle;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}input,textarea,.uneditable-input{width:206px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border lin
 ear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="b
 utton"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #ccc}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.chec
 kbox{min-height:20px;padding-left:20px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[clas
 s*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.
 span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left}.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning .control-label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09
 853}.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error .control-label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error text
 area{color:#b94a48}.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success .control-label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-gro
 up.success textarea{color:#468847}.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad}.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.c
 ontrol-group.info textarea{color:#3a87ad}.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3}.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad}input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-b
 ox-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{display:inline-block;margin-bottom:10px;font-size:0;white-space:nowrap;vertical-align:middle}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-pre
 pend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on
 ,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px}.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .une
 ditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .btn-group:first-child{margin-left:0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input
 -append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-inp
 ut,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.c
 ontrol-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:180px}.form-horizontal .help-block{margin-bottom:0}.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px}.form-horizontal .form-actions{padding-left:180px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:
 100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+
 thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child>th:first-child,.table-bordered tbody:first-child tr:first-child>td:first-child,.table-bordered tbody:first-child tr:first-child>th:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child>th:last-child,.table-bordered tbody:first-child tr:first-child>td:last-child,.table-bordered tbody:first-child tr:first-child>th:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child>th:first-child,.table-bordered tbody:last-child tr:last-child>td:first-child,.table-borde
 red tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child>th:last-child,.table-bordered tbody:last-child tr:last-child>td:last-child,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;border-bottom-left-radius:0;-moz-border-radius-bottomleft:0}.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;border-bottom-right-radius:0;-moz-border-ra
 dius-bottomright:0}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5}table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="sp
 an"]{display:table-cell;float:none;margin-left:0}.table td.span1,.table th.span1{float:none;width:44px;margin-left:0}.table td.span2,.table th.span2{float:none;width:124px;margin-left:0}.table td.span3,.table th.span3{float:none;width:204px;margin-left:0}.table td.span4,.table th.span4{float:none;width:284px;margin-left:0}.table td.span5,.table th.span5{float:none;width:364px;margin-left:0}.table td.span6,.table th.span6{float:none;width:444px;margin-left:0}.table td.span7,.table th.span7{float:none;width:524px;margin-left:0}.table td.span8,.table th.span8{float:none;width:604px;margin-left:0}.table td.span9,.table th.span9{float:none;width:684px;margin-left:0}.table td.span10,.table th.span10{float:none;width:764px;margin-left:0}.table td.span11,.table th.span11{float:none;width:844px;margin-left:0}.table td.span12,.table th.span12{float:none;width:924px;margin-left:0}.table tbody tr.success>td{background-color:#dff0d8}.table tbody tr.error>td{background-color:#f2dede}.table tbody 
 tr.warning>td{background-color:#fcf8e3}.table tbody tr.info>td{background-color:#d9edf7}.table-hover tbody tr.success:hover>td{background-color:#d0e9c6}.table-hover tbody tr.error:hover>td{background-color:#ebcccc}.table-hover tbody tr.warning:hover>td{background-color:#faf2cc}.table-hover tbody tr.info:hover>td{background-color:#c4e3f3}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"]
 ,.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{ba
 ckground-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{back
 ground-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{backg
 round-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-p
 osition:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -9
 6px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{width:16px;background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizont
 al{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fulls
 creen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.drop
 down-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:non
 e;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #00
 0;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover>.dropdown-menu{display:block}.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown-submenu.pull-left{float:none}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-borde
 r-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{z-index:1051;margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative
 ;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 12px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e
 6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*ma
 rgin-left:0}.btn:hover,.btn:focus{color:#333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margi
 n-top:4px}.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0}.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px}.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-moz-linear-gradient(top,#08c,#04c);background-image:-webkit-gradien
 t(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f8940
 6));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));back
 ground-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-w
 ebkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-line
 ar-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-moz-linear-gradient(top,#444,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o
 -linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.bt
 n-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333;text-decoration:none}.btn-group{position:relative;display:inline-block;*display:inline;*margin-left:.3em;font-size:0;white-space:nowrap;vertical-align:middle;*zoom:1}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radiu
 s:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px}.btn-group>.btn-mini{font-size:10.5px}.btn-group>.btn-small{font-size:11.9px}.btn-group>.btn-large{font-size:17.5px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group
 >.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5
 px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-large 
 .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.btn-mini .caret,.btn-small .caret{margin-top:8px}.dropup .btn-large .caret{border-bottom-width:5px}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical>.btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:
 6px 6px 0 0}.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert,.alert h4{color:#c09853}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-success h4{color:#468847}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-danger h4,.alert-error h4{color:#b94a48}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-info h4{color:#3a87ad}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:n
 one}.nav>li>a{display:block}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li>a>img{max-width:none}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover,.nav-list>.active>a:focus{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:bef
 ore,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover,.nav-tabs>li>a:focus{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover,.nav-pills>.active>a:focus{color:#ff
 f;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;borde
 r-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret,.nav li.dropdown.open a:focus .caret{border-top-color:#fff;border-bottom-color
 :#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover,.tabs-below>.nav-tabs>li>a:focus{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>l
 i{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover,.tabs-left>.nav-tabs>li>a:focus{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover,.tabs-right>.nav-tabs>li>a:focus{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs
  .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);*zoom:1;-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1
 px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar-inner:before,.navbar-inner:after{display:table;line-height:0;content:""}.navbar-inner:after{clear:both}.navbar .container{width:auto}.nav-collapse.collapse{height:auto;overflow:visible}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777;text-shadow:0 1px 0 #fff}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px;color:#777}.navbar-link{color:#777}.navbar-link:hover,.navbar-link:focus{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:
 table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:5px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.na
 vbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px}.navbar-fixed-bottom .navbar-inner{border-width:1px 0 0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,0.1);box-shadow:0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,0.1);box-shadow:0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navba
 r .nav.pull-right{float:right;margin-right:0}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-grad
 ient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color
 :#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;
 border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333;border-bottom-color:#333}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777;border-bottom-color:#777}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.n
 avbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#fff}.n
 avbar-inverse .brand{color:#999}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dro
 pdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;te
 xt-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-moz-linear-gradient(top,#151515,#040404);background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:f
 ocus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb>li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb>li>.divider{padding:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination ul>li{display:inline}.pagination ul>li>a,.pagination 
 ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5}.pagination ul>.active>a,.pagination ul>.active>span{color:#999;cursor:default}.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999;cursor:default;background-color:transparent}.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-rig
 ht-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px}.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-bottom-left-radius:3px;border
 -bottom-left-radius:3px;-webkit-border-top-left-radius:3px;border-top-left-radius:3px;-moz-border-radius-bottomleft:3px;-moz-border-radius-topleft:3px}.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;border-bottom-right-radius:3px;-moz-border-radius-topright:3px;-moz-border-radius-bottomright:3px}.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px}.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-b
 order-radius:15px;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;cursor:default;background-color:#fff}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;outline:0;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-cli
 p:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:10%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{position:relative;max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin
 -bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}.tooltip{position:absolute;z-index:1030;display:block;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{padding:5px 0;margin-top:-3px}.tooltip.right{padding:0 5px;margin-left:3px}.tooltip.bottom{padding:5px 0;margin-top:3px}.tooltip.left{padding:0 5px;margin-left:-3px}.tooltip-inner{max-width:200px;padding:8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.
 left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;white-space:normal;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-r
 adius:5px 5px 0 0;-moz-border-radius:5px 5

<TRUNCATED>

[15/15] git commit: initial import.

Posted by cr...@apache.org.
initial import.


Project: http://git-wip-us.apache.org/repos/asf/incubator-samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-samza/commit/5ff71e51
Tree: http://git-wip-us.apache.org/repos/asf/incubator-samza/tree/5ff71e51
Diff: http://git-wip-us.apache.org/repos/asf/incubator-samza/diff/5ff71e51

Branch: refs/heads/master
Commit: 5ff71e51f650c74636b17653fcc5172a653eab65
Parents: 
Author: Chris Riccomini <cr...@criccomi-mn.linkedin.biz>
Authored: Mon Aug 12 09:19:28 2013 -0700
Committer: Chris Riccomini <cr...@criccomi-mn.linkedin.biz>
Committed: Mon Aug 12 09:19:28 2013 -0700

----------------------------------------------------------------------
 .gitignore                                      |   24 +
 DISCLAIMER                                      |    1 +
 HEADER                                          |   16 +
 LICENSE                                         |  580 ++++
 README.md                                       |   55 +
 RELEASE.md                                      |   16 +
 build.gradle                                    |  182 ++
 docs/README.md                                  |   35 +
 docs/_config.yml                                |    5 +
 docs/_layouts/default.html                      |   77 +
 docs/_layouts/page.html                         |    7 +
 docs/_notes/TODO.md                             |   95 +
 docs/_tools/generate-javadocs.sh                |   23 +
 docs/_tools/publish-site.sh                     |   34 +
 docs/community/committers.md                    |   46 +
 docs/community/irc.md                           |   12 +
 docs/community/mailing-lists.md                 |   16 +
 docs/contribute/code.md                         |   16 +
 docs/contribute/coding-guide.md                 |   97 +
 docs/contribute/disclaimer.md                   |    6 +
 docs/contribute/projects.md                     |   20 +
 docs/contribute/rules.md                        |   22 +
 docs/contribute/seps.md                         |   27 +
 docs/css/main.css                               |  161 ++
 .../documentation/comparisons/mupd8-samza.png   |  Bin 0 -> 8217 bytes
 .../learn/documentation/comparisons/mupd8.png   |  Bin 0 -> 7308 bytes
 .../documentation/container/checkpointing-2.png |  Bin 0 -> 18449 bytes
 .../documentation/container/checkpointing.png   |  Bin 0 -> 14613 bytes
 .../learn/documentation/container/job-flow.png  |  Bin 0 -> 31104 bytes
 .../learn/documentation/container/metrics.png   |  Bin 0 -> 20542 bytes
 .../container/tasks-and-partitions.png          |  Bin 0 -> 14362 bytes
 .../documentation/introduction/dag.graffle      | 1009 +++++++
 .../learn/documentation/introduction/dag.png    |  Bin 0 -> 22276 bytes
 .../introduction/group-by-example.png           |  Bin 0 -> 18340 bytes
 .../documentation/introduction/job.graffle      |  512 ++++
 .../learn/documentation/introduction/job.png    |  Bin 0 -> 8954 bytes
 .../introduction/job_detail.graffle             | 1320 +++++++++
 .../documentation/introduction/job_detail.png   |  Bin 0 -> 18176 bytes
 .../introduction/samza-ecosystem.png            |  Bin 0 -> 2473 bytes
 .../documentation/introduction/samza-hadoop.png |  Bin 0 -> 2542 bytes
 .../introduction/samza-yarn-integration.png     |  Bin 0 -> 9400 bytes
 .../samza-yarn-kafka-integration.png            |  Bin 0 -> 12798 bytes
 .../introduction/samza_state.graffle            | 1654 +++++++++++
 .../documentation/introduction/samza_state.png  |  Bin 0 -> 40635 bytes
 .../documentation/introduction/stream.graffle   | 2670 ++++++++++++++++++
 .../learn/documentation/introduction/stream.png |  Bin 0 -> 19806 bytes
 .../motivation/data-processing-spectrum-1.png   |  Bin 0 -> 21773 bytes
 .../motivation/data-processing-spectrum-2.png   |  Bin 0 -> 21902 bytes
 .../motivation/data-processing-spectrum-3.png   |  Bin 0 -> 29389 bytes
 .../documentation/yarn/samza-am-dashboard.png   |  Bin 0 -> 55603 bytes
 docs/img/apache-egg-logo.png                    |  Bin 0 -> 8626 bytes
 docs/img/samza-icon.png                         |  Bin 0 -> 7722 bytes
 docs/index.md                                   |   34 +
 docs/learn/documentation/0.7.0/api/overview.md  |   99 +
 .../0.7.0/comparisons/introduction.md           |   50 +
 .../documentation/0.7.0/comparisons/mupd8.md    |   72 +
 .../documentation/0.7.0/comparisons/storm.md    |   94 +
 .../0.7.0/container/checkpointing.md            |   45 +
 .../documentation/0.7.0/container/event-loop.md |   61 +
 .../documentation/0.7.0/container/index.md      |   18 +
 docs/learn/documentation/0.7.0/container/jmx.md |   13 +
 .../documentation/0.7.0/container/metrics.md    |   50 +
 .../0.7.0/container/state-management.md         |  115 +
 .../documentation/0.7.0/container/streams.md    |   40 +
 .../0.7.0/container/task-runner.md              |   43 +
 .../documentation/0.7.0/container/windowing.md  |   16 +
 docs/learn/documentation/0.7.0/index.html       |   73 +
 .../0.7.0/introduction/architecture.md          |   90 +
 .../0.7.0/introduction/background.md            |   55 +
 .../0.7.0/introduction/concepts.md              |   58 +
 .../0.7.0/jobs/configuration-table.html         |  224 ++
 .../documentation/0.7.0/jobs/configuration.md   |   45 +
 .../documentation/0.7.0/jobs/job-runner.md      |   46 +
 docs/learn/documentation/0.7.0/jobs/logging.md  |   53 +
 .../learn/documentation/0.7.0/jobs/packaging.md |   31 +
 .../learn/documentation/0.7.0/jobs/yarn-jobs.md |   16 +
 .../documentation/0.7.0/operations/kafka.md     |   16 +
 .../documentation/0.7.0/operations/security.md  |   56 +
 .../0.7.0/yarn/application-master.md            |   53 +
 .../learn/documentation/0.7.0/yarn/isolation.md |   30 +
 docs/learn/tutorials/0.7.0/committing.md        |    6 +
 .../tutorials/0.7.0/configuring-kafka-system.md |    6 +
 docs/learn/tutorials/0.7.0/group-by-count.md    |    6 +
 docs/learn/tutorials/0.7.0/index.md             |   17 +
 docs/learn/tutorials/0.7.0/initialize-close.md  |    6 +
 docs/learn/tutorials/0.7.0/joining-streams.md   |    6 +
 docs/learn/tutorials/0.7.0/sort-stream.md       |    6 +
 docs/learn/tutorials/0.7.0/windowing.md         |    6 +
 docs/startup/download/index.md                  |   81 +
 docs/startup/hello-samza/0.7.0/index.md         |   84 +
 gradle.properties                               |    3 +
 gradle/buildscript.gradle                       |   12 +
 gradle/dependency-versions-scala-2.8.1.gradle   |    6 +
 gradle/dependency-versions-scala-2.9.2.gradle   |    6 +
 gradle/dependency-versions.gradle               |   12 +
 gradle/license.gradle                           |    9 +
 gradle/maven.gradle                             |   19 +
 gradle/wrapper/gradle-wrapper.jar               |  Bin 0 -> 49875 bytes
 gradle/wrapper/gradle-wrapper.properties        |    6 +
 gradlew                                         |  164 ++
 .../main/java/org/apache/samza/Partition.java   |   59 +
 .../java/org/apache/samza/SamzaException.java   |   43 +
 .../org/apache/samza/checkpoint/Checkpoint.java |   67 +
 .../samza/checkpoint/CheckpointManager.java     |   34 +
 .../checkpoint/CheckpointManagerFactory.java    |   27 +
 .../java/org/apache/samza/config/Config.java    |  219 ++
 .../apache/samza/config/ConfigException.java    |   38 +
 .../org/apache/samza/config/ConfigFactory.java  |   26 +
 .../org/apache/samza/config/ConfigRewriter.java |   27 +
 .../java/org/apache/samza/config/MapConfig.java |  107 +
 .../org/apache/samza/job/ApplicationStatus.java |   34 +
 .../org/apache/samza/job/CommandBuilder.java    |   62 +
 .../java/org/apache/samza/job/StreamJob.java    |   32 +
 .../org/apache/samza/job/StreamJobFactory.java  |   26 +
 .../java/org/apache/samza/metrics/Counter.java  |   75 +
 .../java/org/apache/samza/metrics/DataType.java |   34 +
 .../java/org/apache/samza/metrics/Gauge.java    |   53 +
 .../java/org/apache/samza/metrics/Metric.java   |   27 +
 .../apache/samza/metrics/MetricsRegistry.java   |   26 +
 .../apache/samza/metrics/MetricsReporter.java   |   28 +
 .../samza/metrics/MetricsReporterFactory.java   |   26 +
 .../org/apache/samza/metrics/MetricsType.java   |   34 +
 .../apache/samza/metrics/MetricsVisitor.java    |   37 +
 .../samza/metrics/ReadableMetricsRegistry.java  |   33 +
 .../ReadableMetricsRegistryListener.java        |   26 +
 .../apache/samza/serializers/Deserializer.java  |   24 +
 .../org/apache/samza/serializers/Serde.java     |   23 +
 .../apache/samza/serializers/SerdeFactory.java  |   26 +
 .../apache/samza/serializers/Serializer.java    |   24 +
 .../org/apache/samza/storage/StorageEngine.java |   49 +
 .../samza/storage/StorageEngineFactory.java     |   61 +
 .../samza/system/IncomingMessageEnvelope.java   |   98 +
 .../system/IncomingMessageEnvelopePicker.java   |   26 +
 .../samza/system/OutgoingMessageEnvelope.java   |  134 +
 .../org/apache/samza/system/SystemAdmin.java    |   28 +
 .../org/apache/samza/system/SystemConsumer.java |   35 +
 .../org/apache/samza/system/SystemFactory.java  |   31 +
 .../org/apache/samza/system/SystemProducer.java |   32 +
 .../org/apache/samza/system/SystemStream.java   |   78 +
 .../samza/system/SystemStreamPartition.java     |   77 +
 .../system/SystemStreamPartitionIterator.java   |   83 +
 .../org/apache/samza/task/ClosableTask.java     |   24 +
 .../org/apache/samza/task/InitableTask.java     |   26 +
 .../org/apache/samza/task/MessageCollector.java |   26 +
 .../java/org/apache/samza/task/StreamTask.java  |   26 +
 .../java/org/apache/samza/task/TaskContext.java |   31 +
 .../org/apache/samza/task/TaskCoordinator.java  |   26 +
 .../samza/task/TaskLifecycleListener.java       |   59 +
 .../task/TaskLifecycleListenerFactory.java      |   26 +
 .../org/apache/samza/task/WindowableTask.java   |   24 +
 .../apache/samza/util/BlockingEnvelopeMap.java  |  251 ++
 .../main/java/org/apache/samza/util/Clock.java  |   24 +
 .../apache/samza/util/NoOpMetricsRegistry.java  |   36 +
 .../samza/util/SinglePartitionSystemAdmin.java  |   46 +
 .../TestSystemStreamPartitionIterator.java      |  118 +
 .../samza/util/TestBlockingEnvelopeMap.java     |  207 ++
 .../samza/util/TestNoOpMetricsRegistry.java     |   55 +
 .../util/TestSinglePartitionSystemAdmin.java    |   37 +
 .../file/FileSystemCheckpointManager.scala      |   86 +
 .../FileSystemCheckpointManagerConfig.scala     |   31 +
 .../org/apache/samza/config/JobConfig.scala     |   43 +
 .../org/apache/samza/config/MetricsConfig.scala |   53 +
 .../apache/samza/config/ScalaMapConfig.scala    |   37 +
 .../apache/samza/config/SerializerConfig.scala  |   42 +
 .../samza/config/ShellCommandConfig.scala       |   53 +
 .../org/apache/samza/config/StorageConfig.scala |   46 +
 .../org/apache/samza/config/StreamConfig.scala  |   80 +
 .../org/apache/samza/config/SystemConfig.scala  |   52 +
 .../org/apache/samza/config/TaskConfig.scala    |   75 +
 .../factories/PropertiesConfigFactory.scala     |   49 +
 .../serializers/JsonConfigSerializer.scala      |   39 +
 .../apache/samza/container/SamzaContainer.scala |  608 ++++
 .../samza/container/SamzaContainerMetrics.scala |   33 +
 .../apache/samza/container/TaskInstance.scala   |  241 ++
 .../samza/container/TaskInstanceMetrics.scala   |   34 +
 .../scala/org/apache/samza/job/JobRunner.scala  |  135 +
 .../apache/samza/job/ShellCommandBuilder.scala  |   39 +
 .../samza/job/local/LocalJobFactory.scala       |   82 +
 .../org/apache/samza/job/local/ProcessJob.scala |  129 +
 .../org/apache/samza/job/local/ThreadJob.scala  |   81 +
 .../org/apache/samza/metrics/JmxServer.scala    |  110 +
 .../org/apache/samza/metrics/JvmMetrics.scala   |  148 +
 .../samza/metrics/MetricsRegistryMap.scala      |   70 +
 .../samza/metrics/reporter/JmxReporter.scala    |  145 +
 .../apache/samza/metrics/reporter/Metrics.scala |   55 +
 .../samza/metrics/reporter/MetricsHeader.scala  |   68 +
 .../metrics/reporter/MetricsSnapshot.scala      |   43 +
 .../reporter/MetricsSnapshotReporter.scala      |  144 +
 .../MetricsSnapshotReporterFactory.scala        |  125 +
 .../samza/serializers/CheckpointSerde.scala     |   66 +
 .../apache/samza/serializers/SerdeManager.scala |  123 +
 .../apache/samza/serializers/StringSerde.scala  |   39 +
 .../samza/storage/TaskStorageManager.scala      |  121 +
 .../org/apache/samza/system/DefaultPicker.scala |   28 +
 .../apache/samza/system/SystemConsumers.scala   |  134 +
 .../apache/samza/system/SystemProducers.scala   |   49 +
 .../apache/samza/task/ReadableCollector.scala   |   37 +
 .../apache/samza/task/ReadableCoordinator.scala |   34 +
 .../apache/samza/util/DaemonThreadFactory.scala |   30 +
 .../main/scala/org/apache/samza/util/Util.scala |  122 +
 samza-core/src/test/resources/test.properties   |   24 +
 .../file/TestFileSystemCheckpointManager.scala  |   67 +
 .../factories/TestPropertiesConfigFactory.scala |   45 +
 .../org/apache/samza/job/TestJobRunner.scala    |   52 +
 .../apache/samza/job/local/TestProcessJob.scala |   44 +
 .../apache/samza/job/local/TestThreadJob.scala  |   49 +
 .../apache/samza/metrics/TestJmxServer.scala    |   58 +
 .../metrics/reporter/TestJmxReporter.scala      |   81 +
 .../samza/task/TestReadableCoordinator.scala    |   45 +
 .../lib/kafka_2.8.1-0.8.1-SNAPSHOT-test.jar     |  Bin 0 -> 878410 bytes
 samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT.jar  |  Bin 0 -> 2680590 bytes
 .../lib/kafka_2.9.2-0.8.1-SNAPSHOT-test.jar     |  Bin 0 -> 906945 bytes
 samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT.jar  |  Bin 0 -> 2747348 bytes
 .../kafka/KafkaCheckpointManager.scala          |  294 ++
 .../kafka/KafkaCheckpointManagerFactory.scala   |   94 +
 .../org/apache/samza/config/KafkaConfig.scala   |   99 +
 .../apache/samza/config/KafkaSerdeConfig.scala  |   36 +
 .../samza/config/RegExTopicGenerator.scala      |  110 +
 .../apache/samza/serializers/KafkaSerde.scala   |   46 +
 .../apache/samza/system/kafka/BrokerProxy.scala |  215 ++
 .../samza/system/kafka/BrokerProxyMetrics.scala |   52 +
 .../samza/system/kafka/DefaultFetch.scala       |   48 +
 .../apache/samza/system/kafka/GetOffset.scala   |   94 +
 .../samza/system/kafka/KafkaSystemAdmin.scala   |   51 +
 .../system/kafka/KafkaSystemConsumer.scala      |  188 ++
 .../samza/system/kafka/KafkaSystemFactory.scala |  107 +
 .../system/kafka/KafkaSystemProducer.scala      |  114 +
 .../apache/samza/system/kafka/MessageSink.scala |   30 +
 .../system/kafka/TopicAndPartitionMetrics.scala |   56 +
 .../samza/system/kafka/TopicMetadataCache.scala |   68 +
 .../org/apache/samza/system/kafka/Toss.scala    |   28 +
 .../util/ClientUtilTopicMetadataStore.scala     |   50 +
 .../scala/org/apache/samza/util/KafkaUtil.scala |   44 +
 .../checkpoint/TestKafkaCheckpointManager.scala |  152 +
 .../samza/config/TestKafkaSerdeConfig.scala     |   39 +
 .../samza/config/TestRegExTopicGenerator.scala  |   87 +
 .../samza/serializers/TestKafkaSerde.scala      |   32 +
 .../samza/system/kafka/TestBrokerProxy.scala    |  178 ++
 .../system/kafka/TestKafkaSystemFactory.scala   |   85 +
 .../system/kafka/TestKafkaSystemProducer.scala  |  155 +
 .../system/kafka/TestTopicMetadataCache.scala   |  121 +
 .../java/org/apache/samza/storage/kv/Entry.java |   46 +
 .../samza/storage/kv/KeyValueIterator.java      |   26 +
 .../apache/samza/storage/kv/KeyValueStore.java  |   74 +
 .../apache/samza/storage/kv/CachedStore.scala   |  161 ++
 .../storage/kv/KeyValueStorageEngine.scala      |   97 +
 .../kv/KeyValueStorageEngineFactory.scala       |   78 +
 .../samza/storage/kv/LevelDbKeyValueStore.scala |  156 +
 .../apache/samza/storage/kv/LoggedStore.scala   |   84 +
 .../storage/kv/SerializedKeyValueStore.scala    |   95 +
 .../samza/storage/kv/TestKeyValueStores.scala   |  172 ++
 .../apache/samza/serializers/JsonSerde.scala    |   38 +
 .../serializers/MetricsSnapshotSerde.scala      |   42 +
 .../samza/serializers/TestJsonSerde.scala       |   33 +
 .../serializers/TestMetricsSnapshotSerde.scala  |   43 +
 samza-shell/src/main/assembly/src.xml           |   31 +
 samza-shell/src/main/bash/kill-yarn-job.sh      |   19 +
 samza-shell/src/main/bash/run-am.sh             |   19 +
 samza-shell/src/main/bash/run-class.sh          |   61 +
 samza-shell/src/main/bash/run-job.sh            |   19 +
 samza-shell/src/main/bash/run-task.sh           |   19 +
 samza-test/java.hprof.txt                       |   65 +
 .../test/integration/SimpleStatefulTask.java    |   35 +
 .../test/integration/StatePerfTestTask.java     |   35 +
 .../samza/test/integration/join/Checker.java    |   71 +
 .../samza/test/integration/join/Emitter.java    |   88 +
 .../test/integration/join/EpochPartitioner.java |   13 +
 .../samza/test/integration/join/Joiner.java     |   93 +
 .../samza/test/integration/join/Watcher.java    |   77 +
 samza-test/src/main/resources/common.properties |   25 +
 .../main/resources/hello-stateful-world.samsa   |   13 +
 .../src/main/resources/join/checker.samsa       |   18 +
 .../src/main/resources/join/emitter.samsa       |   15 +
 samza-test/src/main/resources/join/joiner.samsa |   13 +
 .../src/main/resources/join/watcher.samsa       |   13 +
 samza-test/src/main/resources/log4j.xml         |   21 +
 .../src/main/resources/perf/counter.samsa       |   12 +
 .../test/integration/TestStatefulTask.scala     |  436 +++
 .../scalate/WEB-INF/layouts/default.scaml       |   27 +
 .../resources/scalate/WEB-INF/views/index.scaml |  107 +
 .../resources/scalate/css/bootstrap.min.css     |    9 +
 .../main/resources/scalate/js/bootstrap.min.js  |    6 +
 .../org/apache/samza/config/YarnConfig.scala    |   78 +
 .../apache/samza/job/yarn/ClientHelper.scala    |  201 ++
 .../apache/samza/job/yarn/SamzaAppMaster.scala  |   86 +
 .../job/yarn/SamzaAppMasterLifecycle.scala      |   73 +
 .../samza/job/yarn/SamzaAppMasterMetrics.scala  |   90 +
 .../samza/job/yarn/SamzaAppMasterService.scala  |   66 +
 .../samza/job/yarn/SamzaAppMasterState.scala    |   54 +
 .../job/yarn/SamzaAppMasterTaskManager.scala    |  334 +++
 .../apache/samza/job/yarn/YarnAppMaster.scala   |   76 +
 .../samza/job/yarn/YarnAppMasterListener.scala  |   73 +
 .../org/apache/samza/job/yarn/YarnJob.scala     |  108 +
 .../apache/samza/job/yarn/YarnJobFactory.scala  |   33 +
 .../samza/util/hadoop/HttpFileSystem.scala      |   98 +
 .../samza/util/hadoop/HttpInputStream.scala     |   46 +
 .../webapp/ApplicationMasterRestServlet.scala   |  104 +
 .../webapp/ApplicationMasterWebServlet.scala    |   43 +
 .../org/apache/samza/webapp/WebAppServer.scala  |   51 +
 .../job/yarn/TestSamzaAppMasterLifecycle.scala  |  135 +
 .../job/yarn/TestSamzaAppMasterService.scala    |   49 +
 .../yarn/TestSamzaAppMasterTaskManager.scala    |  431 +++
 .../samza/job/yarn/TestYarnAppMaster.scala      |  143 +
 settings.gradle                                 |    7 +
 304 files changed, 25647 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..ac261dc
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,24 @@
+project/boot/
+target/
+.classpath
+.project
+.scala_dependencies
+.settings/
+dist/
+record_timestamps.log*
+deployable.tgz
+.idea/
+.idea_modules/
+*.iml
+*.ipr
+*.iws
+*/.cache
+dashboard-deployable.tgz
+deployable.tar
+dist-dashboard
+docs/_site
+.gradle
+build
+**/bin
+samza-test/state
+docs/learn/documentation/0.7.0/api/javadocs

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..1ec4f5c
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1 @@
+Apache Samza is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/HEADER
----------------------------------------------------------------------
diff --git a/HEADER b/HEADER
new file mode 100644
index 0000000..60b675e
--- /dev/null
+++ b/HEADER
@@ -0,0 +1,16 @@
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you 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.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..fd83093
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,580 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
+
+-----------------------------------------------------------------------
+
+SLF4J LICENSE
+
+Copyright (c) 2004-2013 QOS.ch
+All rights reserved.
+
+Permission is hereby granted, free  of charge, to any person obtaining
+a  copy  of this  software  and  associated  documentation files  (the
+"Software"), to  deal in  the Software without  restriction, including
+without limitation  the rights to  use, copy, modify,  merge, publish,
+distribute,  sublicense, and/or sell  copies of  the Software,  and to
+permit persons to whom the Software  is furnished to do so, subject to
+the following conditions:
+
+The  above  copyright  notice  and  this permission  notice  shall  be
+included in all copies or substantial portions of the Software.
+
+THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+-----------------------------------------------------------------------
+
+JUNIT LICENSE
+
+JUnit
+
+Common Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+      a) in the case of the initial Contributor, the initial code and
+         documentation distributed under this Agreement, and
+      b) in the case of each subsequent Contributor:
+
+      i) changes to the Program, and
+
+      ii) additions to the Program;
+
+      where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are
+not derivative works of the Program. 
+
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents " mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+      a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and
+such derivative works, in source code and object code form.
+
+      b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under
+Licensed Patents to make, use, sell, offer to sell, import and otherwise
+transfer the Contribution of such Contributor, if any, in source code and
+object code form. This patent license shall apply to the combination of the
+Contribution and the Program if, at the time the Contribution is added by the
+Contributor, such addition of the Contribution causes such combination to be
+covered by the Licensed Patents. The patent license shall not apply to any
+other combinations which include the Contribution. No hardware per se is
+licensed hereunder. 
+
+      c) Recipient understands that although each Contributor grants the
+licenses to its Contributions set forth herein, no assurances are provided by
+any Contributor that the Program does not infringe the patent or other
+intellectual property rights of any other entity. Each Contributor disclaims
+any liability to Recipient for claims brought by any other entity based on
+infringement of intellectual property rights or otherwise. As a condition to
+exercising the rights and licenses granted hereunder, each Recipient hereby
+assumes sole responsibility to secure any other intellectual property rights
+needed, if any. For example, if a third party patent license is required to
+allow Recipient to distribute the Program, it is Recipient's responsibility to
+acquire that license before distributing the Program.
+
+      d) Each Contributor represents that to its knowledge it has sufficient
+copyright rights in its Contribution, if any, to grant the copyright license
+set forth in this Agreement. 
+
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under
+its own license agreement, provided that:
+
+      a) it complies with the terms and conditions of this Agreement; and
+
+      b) its license agreement:
+
+      i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title
+and non-infringement, and implied warranties or conditions of merchantability
+and fitness for a particular purpose; 
+
+      ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits; 
+
+      iii) states that any provisions which differ from this Agreement are
+offered by that Contributor alone and not by any other party; and
+
+      iv) states that source code for the Program is available from such
+Contributor, and informs licensees how to obtain it in a reasonable manner on
+or through a medium customarily used for software exchange. 
+
+When the Program is made available in source code form:
+
+      a) it must be made available under this Agreement; and 
+
+      b) a copy of this Agreement must be included with each copy of the
+Program. 
+
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor
+to control, and cooperate with the Commercial Contributor in, the defense and
+any related settlement negotiations. The Indemnified Contributor may
+participate in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using
+and distributing the Program and assumes all risks associated with its exercise
+of rights under this Agreement, including but not limited to the risks and
+costs of program errors, compliance with applicable laws, damage to or loss of
+data, programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against a Contributor with respect to
+a patent applicable to software (including a cross-claim or counterclaim in a
+lawsuit), then any patent licenses granted by that Contributor to such
+Recipient under this Agreement shall terminate as of the date such litigation
+is filed. In addition, if Recipient institutes patent litigation against any
+entity (including a cross-claim or counterclaim in a lawsuit) alleging that the
+Program itself (excluding combinations of the Program with other software or
+hardware) infringes such Recipient's patent(s), then such Recipient's rights
+granted under Section 2(b) shall terminate as of the date such litigation is
+filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue
+and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+IBM is the initial Agreement Steward. IBM may assign the responsibility to
+serve as the Agreement Steward to a suitable separate entity. Each new version
+of the Agreement will be given a distinguishing version number. The Program
+(including Contributions) may always be distributed subject to the version of
+the Agreement under which it was received. In addition, after a new version of
+the Agreement is published, Contributor may elect to distribute the Program
+(including its Contributions) under the new version. Except as expressly stated
+in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to
+the intellectual property of any Contributor under this Agreement, whether
+expressly, by implication, estoppel or otherwise. All rights in the Program not
+expressly granted under this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial
+in any resulting litigation. 
+
+-----------------------------------------------------------------------
+
+GRIZZLED SLF4J LICENSE
+
+This software is released under a BSD license, adapted from
+<http://opensource.org/licenses/bsd-license.php>
+
+Copyright &copy; 2010 Brian M. Clapper.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice,
+  this list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+* Neither the name "clapper.org" nor the names of its contributors may be
+  used to endorse or promote products derived from this software without
+  specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+POSSIBILITY OF SUCH DAMAGE.
+
+-----------------------------------------------------------------------
+
+SCALATRA LICENSE
+
+Scalatra is distributed under the following terms:
+
+Copyright (c) Alan Dipert <al...@gmail.com>. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+1. Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY AUTHOR AND CONTRIBUTORS ``AS IS'' AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED.  IN NO EVENT SHALL AUTHOR OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+SUCH DAMAGE.
+
+-----------------------------------------------------------------------
+
+SCALA LICENSE
+
+Copyright (c) 2002-2013 EPFL, Lausanne, unless otherwise specified.
+All rights reserved.
+
+This software was developed by the Programming Methods Laboratory of the
+Swiss Federal Institute of Technology (EPFL), Lausanne, Switzerland.
+
+Permission to use, copy, modify, and distribute this software in source
+or binary form for any purpose with or without fee is hereby granted,
+provided that the following conditions are met:
+
+   1. Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+   2. Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+
+   3. Neither the name of the EPFL nor the names of its contributors
+      may be used to endorse or promote products derived from this
+      software without specific prior written permission.
+
+
+THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+SUCH DAMAGE.
+
+-----------------------------------------------------------------------
+
+JOPTSIMPLE LICENSE
+
+The MIT License
+
+Copyright (c) 2004-2013 Paul R. Holser, Jr.
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..d55740e
--- /dev/null
+++ b/README.md
@@ -0,0 +1,55 @@
+## What is Samza?
+
+Apache Incubator Samza is a distributed stream processing framework. It uses <a target="_blank" href="http://kafka.apache.org">Apache Kafka</a> for messaging, and <a target="_blank" href="http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html">Apache Hadoop YARN</a> to provide fault tolerance, processor isolation, security, and resource management.
+
+* **Simpe API:** Unlike most low-level messaging system APIs, Samza provides a very simple call-back based "process message" API that should be familiar to anyone that's used Map/Reduce.
+* **Managed state:** Samza manages snapshotting and restoration of a stream processor's state. Samza will restore a stream processor's state to a snapshot consistent with the processor's last read messages when the processor is restarted.
+* **Fault tolerance:** Samza will work with YARN to restart your stream processor if there is a machine or processor failure.
+* **Durability:** Samza uses Kafka to guarantee that messages will be processed in the order they were written to a partition, and that no messages will ever be lost.
+* **Scalability:** Samza is partitioned and distributed at every level. Kafka provides ordered, partitioned, re-playable, fault-tolerant streams. YARN provides a distributed environment for Samza containers to run in.
+* **Pluggable:** Though Samza works out of the box with Kafka and YARN, Samza provides a pluggable API that lets you run Samza with other messaging systems and execution environments.
+* **Processor isolation:** Samza works with Apache YARN, which supports processor security through Hadoop's security model, and resource isolation through Linux CGroups.
+
+Check out [Hello Samza](/startup/hello-samza/0.7.0) to try Samza. Read the [Background](/learn/documentation/0.7.0/introduction/background.html) page to learn more about Samza.
+
+### Building Samza
+
+To build Samza, run:
+
+    ./gradlew clean build
+
+#### Scala and YARN
+
+Samza builds with [Scala](http://www.scala-lang.org/) 2.9.2 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.0.5-alpha, by default. Use the -PscalaVersion and -PyarnVersion switches to change versions. Samza supports building Scala with 2.8.1 or 2.9.2, and building YARN with 2.0.3-alpha, 2.0.4-alpha, and 2.0.5-alpha.
+
+    ./gradlew -PscalaVersion=2.8.1 -PyarnVersion=2.0.3-alpha clean build
+
+YARN protocols are backwards incompatible, so you must pick the version that matches your YARN grid.
+
+### Testing Samza
+
+To run all tests:
+
+    ./gradlew clean test
+
+To run a single test:
+
+    ./gradlew clean :samza-test:test -Dtest.single=TestStatefulTask
+
+#### Maven
+
+Samza uses Kafka, which is not managed by Maven. To use Kafka as though it were a Maven artifact, Samza installs Kafka into a local repository using the `mvn install` command. You must have Maven installed to build Samza.
+
+### Developers
+
+To get eclipse projects, run:
+
+    ./gradlew eclipse
+
+For IntelliJ, run:
+
+    ./gradlew idea
+
+### Pardon our Dust
+
+Apache Samza is currently undergoing incubation at the [Apache Software Foundation](http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/RELEASE.md
----------------------------------------------------------------------
diff --git a/RELEASE.md b/RELEASE.md
new file mode 100644
index 0000000..06a22df
--- /dev/null
+++ b/RELEASE.md
@@ -0,0 +1,16 @@
+Validate that all Samza source files have proper license information in their header.
+
+    ./gradlew check
+
+Auto-generate all missing headers in files:
+
+    ./gradlew licenseFormatMain
+
+To release to a local Maven repository:
+
+    ./gradlew clean publishToMavenLocal
+    ./gradlew -PscalaVersion=2.8.1 clean publishToMavenLocal
+
+To generate test coverage reports:
+
+    ./gradlew clean jacocoTestReport

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
new file mode 100644
index 0000000..4c4e99c
--- /dev/null
+++ b/build.gradle
@@ -0,0 +1,182 @@
+buildscript {
+  repositories {
+    mavenCentral()
+  }
+
+  apply from: file('gradle/buildscript.gradle'), to: buildscript 
+}
+
+allprojects {
+  repositories {
+    // Required for Kafka. Kafka's 0.8.0-beta1 Maven Central
+    // POM is broken. Should go away in future releases.
+    maven {
+      url 'https://repository.apache.org/content/groups/public'
+    }
+    mavenCentral()
+    mavenLocal()
+  }
+}
+
+apply from: file('gradle/license.gradle')
+apply from: file('gradle/maven.gradle')
+apply from: file("gradle/dependency-versions.gradle")
+apply from: file("gradle/dependency-versions-scala-" + scalaVersion + ".gradle")
+
+subprojects {
+  group = "org.apache.samza"
+
+  apply plugin: 'jacoco'
+  apply plugin: 'eclipse'
+  apply plugin: 'idea'
+  apply plugin: 'project-report'
+}
+
+project(':samza-api') {
+  apply plugin: 'java'
+
+  dependencies {
+    testCompile "junit:junit:$junitVersion"
+  }
+}
+
+project(":samza-core_$scalaVersion") {
+  apply plugin: 'scala'
+
+  dependencies {
+    compile project(':samza-api')
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion"
+    compile "org.codehaus.jackson:jackson-jaxrs:$jacksonVersion"
+    testCompile "junit:junit:$junitVersion"
+  }
+}
+
+project(":samza-kafka_$scalaVersion") {
+  apply plugin: 'scala'
+
+  dependencies {
+    compile project(':samza-api')
+    compile project(":samza-core_$scalaVersion")
+    compile project(":samza-serializers_$scalaVersion")
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "com.101tec:zkclient:$zkClientVersion"
+    compile "org.codehaus.jackson:jackson-jaxrs:$jacksonVersion"
+    // these can all go away when kafka is in maven
+    compile files("lib/kafka_$scalaVersion-" + kafkaVersion + ".jar")
+    compile "com.yammer.metrics:metrics-core:$metricsVersion"
+    compile "com.yammer.metrics:metrics-annotation:$metricsVersion"
+    // end these can all go away when kafka is in maven
+    testCompile "junit:junit:$junitVersion"
+    testCompile "org.mockito:mockito-all:$mockitoVersion"
+    // these can all go away when kafka is in maven
+    testCompile files("lib/kafka_$scalaVersion-$kafkaVersion-test.jar")
+    // end these can all go away when kafka is in maven
+  }
+}
+
+project(":samza-serializers_$scalaVersion") {
+  apply plugin: 'scala'
+
+  dependencies {
+    compile project(':samza-api')
+    compile project(":samza-core_$scalaVersion")
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "org.codehaus.jackson:jackson-jaxrs:$jacksonVersion"
+    testCompile "junit:junit:$junitVersion"
+  }
+}
+
+project(":samza-yarn_$scalaVersion") {
+  apply plugin: 'scala'
+
+  jar {  
+    classifier = "yarn-$yarnVersion"
+  }
+
+  dependencies {
+    compile project(':samza-api')
+    compile project(":samza-core_$scalaVersion")
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.scala-lang:scala-compiler:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "org.codehaus.jackson:jackson-jaxrs:$jacksonVersion"
+    compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion"
+    compile "org.eclipse.jetty:jetty-webapp:$jettyVersion"
+    compile("org.apache.hadoop:hadoop-yarn-api:$yarnVersion") {
+      exclude module: 'slf4j-log4j12'
+    }
+    compile("org.apache.hadoop:hadoop-yarn-common:$yarnVersion") {
+      exclude module: 'slf4j-log4j12'
+    }
+    compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") {
+      exclude module: 'slf4j-log4j12'
+    }
+    compile("org.apache.hadoop:hadoop-common:$yarnVersion") {
+      exclude module: 'slf4j-log4j12'
+      exclude module: 'servlet-api'
+      exclude module: 'jetty'
+      exclude module: 'jetty-util'
+    }
+    compile("org.scalatra:scalatra_$scalaVersion:$scalatraVersion") {
+      exclude module: 'scala-compiler'
+      exclude module: 'slf4j-api'
+    }
+    compile("org.scalatra:scalatra-scalate_$scalaVersion:$scalatraVersion") {
+      exclude module: 'scala-compiler'
+      exclude module: 'slf4j-api'
+    }
+    testCompile "junit:junit:$junitVersion"
+  }
+
+  repositories {
+    maven {
+      url "http://repo.typesafe.com/typesafe/releases"
+    }
+  }
+}
+
+project(":samza-shell") {
+  apply plugin: 'java'
+
+  task shellTarGz(type: Tar) { 
+    compression = Compression.GZIP 
+    classifier = 'dist'
+    from 'src/main/bash'
+  }
+}
+
+project(":samza-kv_$scalaVersion") {
+  apply plugin: 'scala'
+
+  dependencies {
+    compile project(':samza-api')
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "org.fusesource.leveldbjni:leveldbjni-all:$leveldbVersion"
+    testCompile "junit:junit:$junitVersion"
+  }
+}
+
+project(":samza-test_$scalaVersion") {
+  apply plugin: 'scala'
+
+  dependencies {
+    compile project(':samza-api')
+    compile project(":samza-kv_$scalaVersion")
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.clapper:grizzled-slf4j_$scalaVersion:$grizzledVersion"
+    compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion"
+    compile "javax.mail:mail:1.4"
+    compile files("../samza-kafka/lib/kafka_$scalaVersion-" + kafkaVersion + ".jar")
+    testCompile "junit:junit:$junitVersion"
+    testCompile files("../samza-kafka/lib/kafka_$scalaVersion-" + kafkaVersion + "-test.jar")
+    testCompile "com.101tec:zkclient:$zkClientVersion"
+    testCompile project(":samza-core_$scalaVersion")
+    testCompile project(":samza-kafka_$scalaVersion")
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
new file mode 100644
index 0000000..0c1d55c
--- /dev/null
+++ b/docs/README.md
@@ -0,0 +1,35 @@
+## Setup
+
+Samza's documentation uses Jekyll to build a website out of markdown pages. To install Jekyll, run this command:
+
+    sudo gem install jekyll redcarpet
+
+To run the website locally, execute:
+
+    jekyll serve --watch --host 0.0.0.0
+
+To compile the website in the _site directory, execute:
+
+    jekyll build
+
+## Versioning
+
+The "Learn" section of this website is versioned. To add a new version, copy the folder at the version number-level (0.7.0 to 0.8.0, for example).
+
+All links between pages inside a versioned folder should be relative links, not absolute.
+
+## Javadocs
+
+To auto-generate the latest Javadocs, run:
+
+    _tools/generate-javadocs.sh <version>
+
+The version number is the number that will be used in the /docs/learn/documentation/<version>/api/javadocs path.
+
+## Release
+
+To build and publish the website to Samza's Apache SVN repository, run:
+
+    _tools/publish-site.sh 0.7.0 "updating welcome page" criccomini
+
+This command will re-build the Javadocs and website, checkout https://svn.apache.org/repos/asf/incubator/samza/site/ locally, copy the site into the directory, and commit the changes.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
new file mode 100644
index 0000000..bc84c84
--- /dev/null
+++ b/docs/_config.yml
@@ -0,0 +1,5 @@
+permalink: /:categories/:title
+name: Samza
+pygments: true
+markdown: redcarpet
+exclude: ['_notes']

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_layouts/default.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
new file mode 100644
index 0000000..da2f3cd
--- /dev/null
+++ b/docs/_layouts/default.html
@@ -0,0 +1,77 @@
+<html>
+  <head>
+    <title>Samza {% if page.title %}- {{page.title}}{% endif %}</title>
+    <link href="/css/main.css" rel="stylesheet"/>
+    <link href='http://fonts.googleapis.com/css?family=Ropa+Sans' rel='stylesheet' type='text/css'/>
+    <link href="http://netdna.bootstrapcdn.com/font-awesome/3.1.1/css/font-awesome.css" rel="stylesheet"/>
+    <link rel="icon" type="image/png" href="/img/samza-icon.png">
+  </head>
+  <body>
+    <div class="container">
+      <div class="container-inner">
+      <div class="header">
+        <div class="page">
+          <div class="logo">
+            <a href="/" class="logo">samza</a>
+          </div>
+          <div class="top-icons">
+            <a href="/startup/download"><i class="icon-download top-icon"></i></a>
+            <a href="http://git-wip-us.apache.org/repos/asf/incubator-samza.git" target="_blank"><i class="icon-code top-icon" style="font-weight: bold;"></i></a>
+            <a href="https://twitter.com/samzastream" target="_blank"><i class="icon-twitter top-icon"></i></a>
+          </div>
+        </div>
+      </div>
+    
+      <div class="page">
+        <!-- menu -->
+        <div class="menu">
+          <div class="menu-inner">
+            <div class="menu-title"><i class="icon-rocket"></i> Getting Started</div>
+            <div class="menu-category">
+              <a href="/startup/hello-samza/0.7.0">Hello Samza</a><br/>
+              <a href="/startup/download">Download</a><br/>
+            </div>
+            <div class="menu-title"><i class="icon-book"></i> Learn</div>
+            <div class="menu-category">
+              <a href="/learn/documentation/0.7.0">Documentation</a><br/>
+              <a href="/learn/tutorials/0.7.0">Tutorials</a><br/>
+              FAQ<br/>
+              <a href="http://wiki.apache.org/samza">Wiki</a><br/>
+              Papers &amp; Talks<br/>
+            </div>
+            <div class="menu-title"><i class="icon-comments"></i> Community</div>
+            <div class="menu-category">
+              <a href="/community/mailing-lists.html">Mailing Lists</a><br/>
+              <a href="/community/irc.html">IRC</a><br/>
+              <a href="https://issues.apache.org/jira/browse/SAMZA">Bugs</a><br/>
+              Powered by<br/>
+              Ecosystem<br/>
+              <a href="/community/committers.html">Committers</a>
+            </div>
+            <div class="menu-title"><i class="icon-code"></i> Contribute</div>
+            <div class="menu-category">
+              <a href="/contribute/rules.html">Rules</a><br/>
+              <a href="/contribute/coding-guide.html">Coding Guide</a><br/>
+              <a href="/contribute/projects.html">Projects</a><br/>
+              <a href="/contribute/seps.html">SEPs</a><br/>
+              <a href="/contribute/code.html">Code</a><br/>
+              <a href="https://reviews.apache.org/groups/samza">Review Board</a><br/>
+              <a href="https://builds.apache.org/">Unit Tests</a><br/>
+              <a href="/contribute/disclaimer.html">Disclaimer</a>
+            </div>
+          </div>
+        </div>
+        <!-- body -->
+        <div class="body">
+          {{content}}
+        </div>
+        </div>
+      </div>
+    </div>
+    <div class="footer">
+      <div class="page footer-content">
+        <!-- nothing for now. -->
+      </div>
+    </div>
+  </body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_layouts/page.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/page.html b/docs/_layouts/page.html
new file mode 100644
index 0000000..007adf9
--- /dev/null
+++ b/docs/_layouts/page.html
@@ -0,0 +1,7 @@
+---
+layout: default
+---
+
+<h2>{{ page.title }}</h2>
+
+{{ content }}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_notes/TODO.md
----------------------------------------------------------------------
diff --git a/docs/_notes/TODO.md b/docs/_notes/TODO.md
new file mode 100644
index 0000000..5ce8301
--- /dev/null
+++ b/docs/_notes/TODO.md
@@ -0,0 +1,95 @@
+## TODO items
+
+* rename everything to samza
+* cut a 0.7 branch
+* release samza 0.7.0 snapshot to artifactory
+* docs
+  YARN
+    Fault tolerance
+    Security
+  * comparisons pages
+    * JMS
+    * Aurora
+    * JMS
+    * S4
+  * configuration update (make sure we have everything)
+  * add versioning link in docs page, so you can get to other version numbers
+* tutorials
+  * Configuring a Kafka System
+  * Configuring a YARN cluster
+  * Joining Streams
+  * Sorting a Stream
+  * Group-by and Counting
+  * Initializing and Closing
+  * Windowing
+  * Committing
+* add versioning link in docs page, so you can get to other version numbers
+
+## Rules/coding guideline/config brainstorm
+
+Before contributing to Samza, please have a look at the rules listed below.
+
+### Coding Style
+
+TODO add coding style rules for Java and Scala. Should look into enforcing them through Maven.
+
+### Metrics
+
+TODO we should add some metric rules here.
+
+### Configuration
+
+12. TODO When in Scala * how to handle null/none?
+
+```
+public interface MyConfig
+{
+  @Config("yarn.package.uri")
+  URI getYarnPackageUri();
+
+  @Config("systems.<system>.consumer.factory")
+  @Default()
+  Class<StreamConsumerFactory> getSystemsConsumerFactory(String systemName);
+
+  @Config("systems.<system>.*")
+  Map<String, String> getSystems(String systemName);
+}
+```
+
+<!-- https://github.com/brianm/config-magic -->
+
+Open:
+
+* We are mixing wiring and configuration together. How do other systems handle this?
+* We have fragmented configuration (anybody can define configuration). How do other systems handle this?
+* Want to do "best practices" configuration name validation at compile time (enforce rules outlined above).
+* How to handle getting of submaps? Should it return a config object, instead? If so, should it be typed? (e.g. config.getSubset("foo.bar.") = Config or config.getSubset("foo.bar.") = SystemConfig)
+
+Solved:
+
+* Want to auto-generate documentation based off of configuration.
+  @Description
+* Should support global defaults for a config property. Right now, we do config.getFoo.getOrElse() everywhere.
+  @Default
+* How to handle undefined configuration? How to make this interoperable with both Java and Scala (i.e. should we support Option in Scala)? 
+  getInt("foo", null) or getInt("foo"); latter throws an exception if undefined. In Scala, Option(getInt("foo", null)) does the trick.
+* Should work with CFG2 and Java .properties files.
+* Should remain immutable.
+* Should remove implicits. Just confusing.
+  val kafkaConfig = new KafkaConfig(config)
+  val systemConfig = new SystemConfig(config)
+
+```
+public interface MyConfig
+{
+  @Config("yarn.package.uri")
+  URI getYarnPackageUri();
+
+  @Config("systems.<system>.consumer.factory")
+  @Default()
+  Class<StreamConsumerFactory> getSystemsConsumerFactory(String systemName);
+
+  @Config("systems.<system>.*")
+  Map<String, String> getSystems(String systemName);
+}
+```

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_tools/generate-javadocs.sh
----------------------------------------------------------------------
diff --git a/docs/_tools/generate-javadocs.sh b/docs/_tools/generate-javadocs.sh
new file mode 100755
index 0000000..4beb55e
--- /dev/null
+++ b/docs/_tools/generate-javadocs.sh
@@ -0,0 +1,23 @@
+#!/bin/bash
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+BASE_DIR=$DIR/../..
+VERSION=$1
+JAVADOC_DIR=$BASE_DIR/docs/learn/documentation/$VERSION/api/javadocs
+
+if test -z "$VERSION"; then
+  echo
+  echo "  USAGE:"
+  echo
+  echo "    ${BASH_SOURCE[0]##*/} 0.7.0"
+  echo
+  exit 0
+fi
+
+cd $BASE_DIR
+./gradlew javadoc
+rm -rf $JAVADOC_DIR
+mkdir -p $JAVADOC_DIR
+cp -r $BASE_DIR/samza-api/build/docs/javadoc/* $JAVADOC_DIR
+git add $JAVADOC_DIR
+cd -

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/_tools/publish-site.sh
----------------------------------------------------------------------
diff --git a/docs/_tools/publish-site.sh b/docs/_tools/publish-site.sh
new file mode 100755
index 0000000..285c9e7
--- /dev/null
+++ b/docs/_tools/publish-site.sh
@@ -0,0 +1,34 @@
+#!/bin/bash
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+BASE_DIR=$DIR/../..
+DOCS_DIR=$BASE_DIR/docs
+VERSION=$1
+COMMENT=$2
+USER=$3
+
+if test -z "$VERSION" || test -z "$COMMENT" || test -z "$USER"; then
+  echo
+  echo "  USAGE:"
+  echo
+  echo "    ${BASH_SOURCE[0]##*/} 0.7.0 \"updating welcome page\" criccomini"
+  echo
+  exit 0
+fi
+
+echo "Using uer: $USER"
+echo "Using version: $VERSION"
+echo "Using comment: $COMMENT"
+echo "Generating javadocs."
+$DOCS_DIR/_tools/generate-javadocs.sh $VERSION
+
+echo "Building site."
+cd $DOCS_DIR
+jekyll build
+
+echo "Checking out SVN site."
+SVN_TMP=`mktemp -d /tmp/samza-svn.XXXX`
+svn co https://svn.apache.org/repos/asf/incubator/samza/ $SVN_TMP
+cp -r _site/* $SVN_TMP/site/
+svn add --force $SVN_TMP/site
+svn commit $SVN_TMP -m"$COMMENT" --username $USER
+rm -rf $SVN_TMP

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/community/committers.md
----------------------------------------------------------------------
diff --git a/docs/community/committers.md b/docs/community/committers.md
new file mode 100644
index 0000000..a240978
--- /dev/null
+++ b/docs/community/committers.md
@@ -0,0 +1,46 @@
+---
+layout: page
+title: Committers
+---
+
+**Jakob Homan**<br/>
+Apache Member, Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/in/jghoman" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/blueboxtraveler" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Jay Kreps**<br/>
+Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/in/jaykreps" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/jaykreps" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Chris Riccomini**<br/>
+Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/in/riccomini" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/criccomini" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Sriram Subramanian**<br/>
+Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/pub/sriram-subramanian/3/52a/162" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/sriramsub1" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+### Mentors
+
+**Chris Douglas**<br/>
+Apache Member<br/>
+<a href="http://www.linkedin.com/pub/chris-douglas/1/33a/733" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/chris_douglas" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Jakob Homan**<br/>
+Apache Member, Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/in/jghoman" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/blueboxtraveler" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Arun C Murthy**<br/>
+Apache Member<br/>
+<a href="http://www.linkedin.com/in/acmurthy" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/acmurthy" target="_blank"><i class="icon-twitter committer-icon"></i></a>
+
+**Roman Shaposhnik**<br/>
+Apache Incubator PMC Member<br/>
+<a href="http://www.linkedin.com/in/shaposhnik" target="_blank"><i class="icon-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/rhatr" target="_blank"><i class="icon-twitter committer-icon"></i></a>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/community/irc.md
----------------------------------------------------------------------
diff --git a/docs/community/irc.md b/docs/community/irc.md
new file mode 100644
index 0000000..ac191e5
--- /dev/null
+++ b/docs/community/irc.md
@@ -0,0 +1,12 @@
+---
+layout: page
+title: IRC
+---
+
+We have an IRC channel where there is often a few people hanging around if you want an interactive discussion. You can find us on chat.freenode.net	in #samza room.
+
+### WebIRC
+
+To talk to us right now, just enter a username, and set the channel to #samza, below.
+
+<iframe src="http://webchat.freenode.net/?channels=channel_name" style="width: 100%; height: 500px; border: none;"></iframe>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/community/mailing-lists.md
----------------------------------------------------------------------
diff --git a/docs/community/mailing-lists.md b/docs/community/mailing-lists.md
new file mode 100644
index 0000000..b44dadd
--- /dev/null
+++ b/docs/community/mailing-lists.md
@@ -0,0 +1,16 @@
+---
+layout: page
+title: Mailing Lists
+---
+
+<!-- TODO update with links to real mailing lists when apache gives them to us. -->
+
+<!-- TODO update searchable links when we have them (see kafka page). -->
+
+We have a few mailing lists hosted by Apache:
+
+* **dev@samza.incubator.apache.org**  <small>([subscribe](mailto:dev-subscribe@samza.incubator.apache.org), [unsubscribe](mailto:dev-unsubscribe@samza.incubator.apache.org))</small><br/>
+  A list for discussion on Samza development. Email here for help.
+
+* **commits@samza.incubator.apache.org** <small>([subscribe](mailto:commits-subscribe@samza.incubator.apache.org), [unsubscribe](mailto:commits-unsubscribe@samza.incubator.apache.org))</small><br/>
+  A list to track Samza commits.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/contribute/code.md
----------------------------------------------------------------------
diff --git a/docs/contribute/code.md b/docs/contribute/code.md
new file mode 100644
index 0000000..f580969
--- /dev/null
+++ b/docs/contribute/code.md
@@ -0,0 +1,16 @@
+---
+layout: page
+title: Code
+---
+
+Samza's code is in an Apache Git repository located [here](https://git-wip-us.apache.org/repos/asf?p=incubator-samza.git;a=tree).
+
+You can check out Samza's code by running:
+
+```
+git clone http://git-wip-us.apache.org/repos/asf/incubator-samza.git
+````
+
+If you are a committer you need to use https instead of http to check in.
+
+Please see the [Rules](rules.html) page for information on how to contribute.


[02/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/resources/scalate/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/resources/scalate/js/bootstrap.min.js b/samza-yarn/src/main/resources/scalate/js/bootstrap.min.js
new file mode 100644
index 0000000..bafe4b8
--- /dev/null
+++ b/samza-yarn/src/main/resources/scalate/js/bootstrap.min.js
@@ -0,0 +1,6 @@
+/*!
+* Bootstrap.js by @fat & @mdo
+* Copyright 2012 Twitter, Inc.
+* http://www.apache.org/licenses/LICENSE-2.0.txt
+*/
+!function(e){"use strict";e(function(){e.support.transition=function(){var e=function(){var e=document.createElement("bootstrap"),t={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"},n;for(n in t)if(e.style[n]!==undefined)return t[n]}();return e&&{end:e}}()})}(window.jQuery),!function(e){"use strict";var t='[data-dismiss="alert"]',n=function(n){e(n).on("click",t,this.close)};n.prototype.close=function(t){function s(){i.trigger("closed").remove()}var n=e(this),r=n.attr("data-target"),i;r||(r=n.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,"")),i=e(r),t&&t.preventDefault(),i.length||(i=n.hasClass("alert")?n:n.parent()),i.trigger(t=e.Event("close"));if(t.isDefaultPrevented())return;i.removeClass("in"),e.support.transition&&i.hasClass("fade")?i.on(e.support.transition.end,s):s()};var r=e.fn.alert;e.fn.alert=function(t){return this.each(function(){var r=e(this),i=r.data("alert");i||r.data("alert",i=
 new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.alert.Constructor=n,e.fn.alert.noConflict=function(){return e.fn.alert=r,this},e(document).on("click.alert.data-api",t,n.prototype.close)}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.button.defaults,n)};t.prototype.setState=function(e){var t="disabled",n=this.$element,r=n.data(),i=n.is("input")?"val":"html";e+="Text",r.resetText||n.data("resetText",n[i]()),n[i](r[e]||this.options[e]),setTimeout(function(){e=="loadingText"?n.addClass(t).attr(t,t):n.removeClass(t).removeAttr(t)},0)},t.prototype.toggle=function(){var e=this.$element.closest('[data-toggle="buttons-radio"]');e&&e.find(".active").removeClass("active"),this.$element.toggleClass("active")};var n=e.fn.button;e.fn.button=function(n){return this.each(function(){var r=e(this),i=r.data("button"),s=typeof n=="object"&&n;i||r.data("button",i=new t(this,s)),n=="toggle"?i.toggle():n&&i.setState(n)})},e.fn.button.de
 faults={loadingText:"loading..."},e.fn.button.Constructor=t,e.fn.button.noConflict=function(){return e.fn.button=n,this},e(document).on("click.button.data-api","[data-toggle^=button]",function(t){var n=e(t.target);n.hasClass("btn")||(n=n.closest(".btn")),n.button("toggle")})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.$indicators=this.$element.find(".carousel-indicators"),this.options=n,this.options.pause=="hover"&&this.$element.on("mouseenter",e.proxy(this.pause,this)).on("mouseleave",e.proxy(this.cycle,this))};t.prototype={cycle:function(t){return t||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(e.proxy(this.next,this),this.options.interval)),this},getActiveIndex:function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},to:function(t){var n=this.getActiveIndex(),r=this;if(t>thi
 s.$items.length-1||t<0)return;return this.sliding?this.$element.one("slid",function(){r.to(t)}):n==t?this.pause().cycle():this.slide(t>n?"next":"prev",e(this.$items[t]))},pause:function(t){return t||(this.paused=!0),this.$element.find(".next, .prev").length&&e.support.transition.end&&(this.$element.trigger(e.support.transition.end),this.cycle(!0)),clearInterval(this.interval),this.interval=null,this},next:function(){if(this.sliding)return;return this.slide("next")},prev:function(){if(this.sliding)return;return this.slide("prev")},slide:function(t,n){var r=this.$element.find(".item.active"),i=n||r[t](),s=this.interval,o=t=="next"?"left":"right",u=t=="next"?"first":"last",a=this,f;this.sliding=!0,s&&this.pause(),i=i.length?i:this.$element.find(".item")[u](),f=e.Event("slide",{relatedTarget:i[0],direction:o});if(i.hasClass("active"))return;this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var t=e(a.$indicators.children(
 )[a.getActiveIndex()]);t&&t.addClass("active")}));if(e.support.transition&&this.$element.hasClass("slide")){this.$element.trigger(f);if(f.isDefaultPrevented())return;i.addClass(t),i[0].offsetWidth,r.addClass(o),i.addClass(o),this.$element.one(e.support.transition.end,function(){i.removeClass([t,o].join(" ")).addClass("active"),r.removeClass(["active",o].join(" ")),a.sliding=!1,setTimeout(function(){a.$element.trigger("slid")},0)})}else{this.$element.trigger(f);if(f.isDefaultPrevented())return;r.removeClass("active"),i.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return s&&this.cycle(),this}};var n=e.fn.carousel;e.fn.carousel=function(n){return this.each(function(){var r=e(this),i=r.data("carousel"),s=e.extend({},e.fn.carousel.defaults,typeof n=="object"&&n),o=typeof n=="string"?n:s.slide;i||r.data("carousel",i=new t(this,s)),typeof n=="number"?i.to(n):o?i[o]():s.interval&&i.pause().cycle()})},e.fn.carousel.defaults={interval:5e3,pause:"hover"},e.fn.carousel.Const
 ructor=t,e.fn.carousel.noConflict=function(){return e.fn.carousel=n,this},e(document).on("click.carousel.data-api","[data-slide], [data-slide-to]",function(t){var n=e(this),r,i=e(n.attr("data-target")||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,"")),s=e.extend({},i.data(),n.data()),o;i.carousel(s),(o=n.attr("data-slide-to"))&&i.data("carousel").pause().to(o).cycle(),t.preventDefault()})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.collapse.defaults,n),this.options.parent&&(this.$parent=e(this.options.parent)),this.options.toggle&&this.toggle()};t.prototype={constructor:t,dimension:function(){var e=this.$element.hasClass("width");return e?"width":"height"},show:function(){var t,n,r,i;if(this.transitioning||this.$element.hasClass("in"))return;t=this.dimension(),n=e.camelCase(["scroll",t].join("-")),r=this.$parent&&this.$parent.find("> .accordion-group > .in");if(r&&r.length){i=r.data("collapse");if(i&&i.transitionin
 g)return;r.collapse("hide"),i||r.data("collapse",null)}this.$element[t](0),this.transition("addClass",e.Event("show"),"shown"),e.support.transition&&this.$element[t](this.$element[0][n])},hide:function(){var t;if(this.transitioning||!this.$element.hasClass("in"))return;t=this.dimension(),this.reset(this.$element[t]()),this.transition("removeClass",e.Event("hide"),"hidden"),this.$element[t](0)},reset:function(e){var t=this.dimension();return this.$element.removeClass("collapse")[t](e||"auto")[0].offsetWidth,this.$element[e!==null?"addClass":"removeClass"]("collapse"),this},transition:function(t,n,r){var i=this,s=function(){n.type=="show"&&i.reset(),i.transitioning=0,i.$element.trigger(r)};this.$element.trigger(n);if(n.isDefaultPrevented())return;this.transitioning=1,this.$element[t]("in"),e.support.transition&&this.$element.hasClass("collapse")?this.$element.one(e.support.transition.end,s):s()},toggle:function(){this[this.$element.hasClass("in")?"hide":"show"]()}};var n=e.fn.collapse
 ;e.fn.collapse=function(n){return this.each(function(){var r=e(this),i=r.data("collapse"),s=e.extend({},e.fn.collapse.defaults,r.data(),typeof n=="object"&&n);i||r.data("collapse",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.collapse.defaults={toggle:!0},e.fn.collapse.Constructor=t,e.fn.collapse.noConflict=function(){return e.fn.collapse=n,this},e(document).on("click.collapse.data-api","[data-toggle=collapse]",function(t){var n=e(this),r,i=n.attr("data-target")||t.preventDefault()||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,""),s=e(i).data("collapse")?"toggle":n.data();n[e(i).hasClass("in")?"addClass":"removeClass"]("collapsed"),e(i).collapse(s)})}(window.jQuery),!function(e){"use strict";function r(){e(t).each(function(){i(e(this)).removeClass("open")})}function i(t){var n=t.attr("data-target"),r;n||(n=t.attr("href"),n=n&&/#/.test(n)&&n.replace(/.*(?=#[^\s]*$)/,"")),r=n&&e(n);if(!r||!r.length)r=t.parent();return r}var t="[data-toggle=dropdown]",n=function(t){var n=e(t).o
 n("click.dropdown.data-api",this.toggle);e("html").on("click.dropdown.data-api",function(){n.parent().removeClass("open")})};n.prototype={constructor:n,toggle:function(t){var n=e(this),s,o;if(n.is(".disabled, :disabled"))return;return s=i(n),o=s.hasClass("open"),r(),o||s.toggleClass("open"),n.focus(),!1},keydown:function(n){var r,s,o,u,a,f;if(!/(38|40|27)/.test(n.keyCode))return;r=e(this),n.preventDefault(),n.stopPropagation();if(r.is(".disabled, :disabled"))return;u=i(r),a=u.hasClass("open");if(!a||a&&n.keyCode==27)return n.which==27&&u.find(t).focus(),r.click();s=e("[role=menu] li:not(.divider):visible a",u);if(!s.length)return;f=s.index(s.filter(":focus")),n.keyCode==38&&f>0&&f--,n.keyCode==40&&f<s.length-1&&f++,~f||(f=0),s.eq(f).focus()}};var s=e.fn.dropdown;e.fn.dropdown=function(t){return this.each(function(){var r=e(this),i=r.data("dropdown");i||r.data("dropdown",i=new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.dropdown.Constructor=n,e.fn.dropdown.noConflict=function()
 {return e.fn.dropdown=s,this},e(document).on("click.dropdown.data-api",r).on("click.dropdown.data-api",".dropdown form",function(e){e.stopPropagation()}).on("click.dropdown-menu",function(e){e.stopPropagation()}).on("click.dropdown.data-api",t,n.prototype.toggle).on("keydown.dropdown.data-api",t+", [role=menu]",n.prototype.keydown)}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=n,this.$element=e(t).delegate('[data-dismiss="modal"]',"click.dismiss.modal",e.proxy(this.hide,this)),this.options.remote&&this.$element.find(".modal-body").load(this.options.remote)};t.prototype={constructor:t,toggle:function(){return this[this.isShown?"hide":"show"]()},show:function(){var t=this,n=e.Event("show");this.$element.trigger(n);if(this.isShown||n.isDefaultPrevented())return;this.isShown=!0,this.escape(),this.backdrop(function(){var n=e.support.transition&&t.$element.hasClass("fade");t.$element.parent().length||t.$element.appendTo(document.body),t.$element.show(),n&&t.$e
 lement[0].offsetWidth,t.$element.addClass("in").attr("aria-hidden",!1),t.enforceFocus(),n?t.$element.one(e.support.transition.end,function(){t.$element.focus().trigger("shown")}):t.$element.focus().trigger("shown")})},hide:function(t){t&&t.preventDefault();var n=this;t=e.Event("hide"),this.$element.trigger(t);if(!this.isShown||t.isDefaultPrevented())return;this.isShown=!1,this.escape(),e(document).off("focusin.modal"),this.$element.removeClass("in").attr("aria-hidden",!0),e.support.transition&&this.$element.hasClass("fade")?this.hideWithTransition():this.hideModal()},enforceFocus:function(){var t=this;e(document).on("focusin.modal",function(e){t.$element[0]!==e.target&&!t.$element.has(e.target).length&&t.$element.focus()})},escape:function(){var e=this;this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.modal",function(t){t.which==27&&e.hide()}):this.isShown||this.$element.off("keyup.dismiss.modal")},hideWithTransition:function(){var t=this,n=setTimeout(function(){t.
 $element.off(e.support.transition.end),t.hideModal()},500);this.$element.one(e.support.transition.end,function(){clearTimeout(n),t.hideModal()})},hideModal:function(){var e=this;this.$element.hide(),this.backdrop(function(){e.removeBackdrop(),e.$element.trigger("hidden")})},removeBackdrop:function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},backdrop:function(t){var n=this,r=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var i=e.support.transition&&r;this.$backdrop=e('<div class="modal-backdrop '+r+'" />').appendTo(document.body),this.$backdrop.click(this.options.backdrop=="static"?e.proxy(this.$element[0].focus,this.$element[0]):e.proxy(this.hide,this)),i&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in");if(!t)return;i?this.$backdrop.one(e.support.transition.end,t):t()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),e.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(e.support.tra
 nsition.end,t):t()):t&&t()}};var n=e.fn.modal;e.fn.modal=function(n){return this.each(function(){var r=e(this),i=r.data("modal"),s=e.extend({},e.fn.modal.defaults,r.data(),typeof n=="object"&&n);i||r.data("modal",i=new t(this,s)),typeof n=="string"?i[n]():s.show&&i.show()})},e.fn.modal.defaults={backdrop:!0,keyboard:!0,show:!0},e.fn.modal.Constructor=t,e.fn.modal.noConflict=function(){return e.fn.modal=n,this},e(document).on("click.modal.data-api",'[data-toggle="modal"]',function(t){var n=e(this),r=n.attr("href"),i=e(n.attr("data-target")||r&&r.replace(/.*(?=#[^\s]+$)/,"")),s=i.data("modal")?"toggle":e.extend({remote:!/#/.test(r)&&r},i.data(),n.data());t.preventDefault(),i.modal(s).one("hide",function(){n.focus()})})}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("tooltip",e,t)};t.prototype={constructor:t,init:function(t,n,r){var i,s,o,u,a;this.type=t,this.$element=e(n),this.options=this.getOptions(r),this.enabled=!0,o=this.options.trigger.split(" ");for(a=o
 .length;a--;)u=o[a],u=="click"?this.$element.on("click."+this.type,this.options.selector,e.proxy(this.toggle,this)):u!="manual"&&(i=u=="hover"?"mouseenter":"focus",s=u=="hover"?"mouseleave":"blur",this.$element.on(i+"."+this.type,this.options.selector,e.proxy(this.enter,this)),this.$element.on(s+"."+this.type,this.options.selector,e.proxy(this.leave,this)));this.options.selector?this._options=e.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},getOptions:function(t){return t=e.extend({},e.fn[this.type].defaults,this.$element.data(),t),t.delay&&typeof t.delay=="number"&&(t.delay={show:t.delay,hide:t.delay}),t},enter:function(t){var n=e.fn[this.type].defaults,r={},i;this._options&&e.each(this._options,function(e,t){n[e]!=t&&(r[e]=t)},this),i=e(t.currentTarget)[this.type](r).data(this.type);if(!i.options.delay||!i.options.delay.show)return i.show();clearTimeout(this.timeout),i.hoverState="in",this.timeout=setTimeout(function(){i.hoverState=="in"&&i.show()},i.optio
 ns.delay.show)},leave:function(t){var n=e(t.currentTarget)[this.type](this._options).data(this.type);this.timeout&&clearTimeout(this.timeout);if(!n.options.delay||!n.options.delay.hide)return n.hide();n.hoverState="out",this.timeout=setTimeout(function(){n.hoverState=="out"&&n.hide()},n.options.delay.hide)},show:function(){var t,n,r,i,s,o,u=e.Event("show");if(this.hasContent()&&this.enabled){this.$element.trigger(u);if(u.isDefaultPrevented())return;t=this.tip(),this.setContent(),this.options.animation&&t.addClass("fade"),s=typeof this.options.placement=="function"?this.options.placement.call(this,t[0],this.$element[0]):this.options.placement,t.detach().css({top:0,left:0,display:"block"}),this.options.container?t.appendTo(this.options.container):t.insertAfter(this.$element),n=this.getPosition(),r=t[0].offsetWidth,i=t[0].offsetHeight;switch(s){case"bottom":o={top:n.top+n.height,left:n.left+n.width/2-r/2};break;case"top":o={top:n.top-i,left:n.left+n.width/2-r/2};break;case"left":o={top
 :n.top+n.height/2-i/2,left:n.left-r};break;case"right":o={top:n.top+n.height/2-i/2,left:n.left+n.width}}this.applyPlacement(o,s),this.$element.trigger("shown")}},applyPlacement:function(e,t){var n=this.tip(),r=n[0].offsetWidth,i=n[0].offsetHeight,s,o,u,a;n.offset(e).addClass(t).addClass("in"),s=n[0].offsetWidth,o=n[0].offsetHeight,t=="top"&&o!=i&&(e.top=e.top+i-o,a=!0),t=="bottom"||t=="top"?(u=0,e.left<0&&(u=e.left*-2,e.left=0,n.offset(e),s=n[0].offsetWidth,o=n[0].offsetHeight),this.replaceArrow(u-r+s,s,"left")):this.replaceArrow(o-i,o,"top"),a&&n.offset(e)},replaceArrow:function(e,t,n){this.arrow().css(n,e?50*(1-e/t)+"%":"")},setContent:function(){var e=this.tip(),t=this.getTitle();e.find(".tooltip-inner")[this.options.html?"html":"text"](t),e.removeClass("fade in top bottom left right")},hide:function(){function i(){var t=setTimeout(function(){n.off(e.support.transition.end).detach()},500);n.one(e.support.transition.end,function(){clearTimeout(t),n.detach()})}var t=this,n=this.tip
 (),r=e.Event("hide");this.$element.trigger(r);if(r.isDefaultPrevented())return;return n.removeClass("in"),e.support.transition&&this.$tip.hasClass("fade")?i():n.detach(),this.$element.trigger("hidden"),this},fixTitle:function(){var e=this.$element;(e.attr("title")||typeof e.attr("data-original-title")!="string")&&e.attr("data-original-title",e.attr("title")||"").attr("title","")},hasContent:function(){return this.getTitle()},getPosition:function(){var t=this.$element[0];return e.extend({},typeof t.getBoundingClientRect=="function"?t.getBoundingClientRect():{width:t.offsetWidth,height:t.offsetHeight},this.$element.offset())},getTitle:function(){var e,t=this.$element,n=this.options;return e=t.attr("data-original-title")||(typeof n.title=="function"?n.title.call(t[0]):n.title),e},tip:function(){return this.$tip=this.$tip||e(this.options.template)},arrow:function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},validate:function(){this.$element[0].parentNode||(this.h
 ide(),this.$element=null,this.options=null)},enable:function(){this.enabled=!0},disable:function(){this.enabled=!1},toggleEnabled:function(){this.enabled=!this.enabled},toggle:function(t){var n=t?e(t.currentTarget)[this.type](this._options).data(this.type):this;n.tip().hasClass("in")?n.hide():n.show()},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}};var n=e.fn.tooltip;e.fn.tooltip=function(n){return this.each(function(){var r=e(this),i=r.data("tooltip"),s=typeof n=="object"&&n;i||r.data("tooltip",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.tooltip.Constructor=t,e.fn.tooltip.defaults={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},e.fn.tooltip.noConflict=function(){return e.fn.tooltip=n,this}}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("popover",e,t)};t.pro
 totype=e.extend({},e.fn.tooltip.Constructor.prototype,{constructor:t,setContent:function(){var e=this.tip(),t=this.getTitle(),n=this.getContent();e.find(".popover-title")[this.options.html?"html":"text"](t),e.find(".popover-content")[this.options.html?"html":"text"](n),e.removeClass("fade top bottom left right in")},hasContent:function(){return this.getTitle()||this.getContent()},getContent:function(){var e,t=this.$element,n=this.options;return e=(typeof n.content=="function"?n.content.call(t[0]):n.content)||t.attr("data-content"),e},tip:function(){return this.$tip||(this.$tip=e(this.options.template)),this.$tip},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}});var n=e.fn.popover;e.fn.popover=function(n){return this.each(function(){var r=e(this),i=r.data("popover"),s=typeof n=="object"&&n;i||r.data("popover",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.popover.Constructor=t,e.fn.popover.defaults=e.extend({},e.fn.tooltip.defaults,{placement:"
 right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),e.fn.popover.noConflict=function(){return e.fn.popover=n,this}}(window.jQuery),!function(e){"use strict";function t(t,n){var r=e.proxy(this.process,this),i=e(t).is("body")?e(window):e(t),s;this.options=e.extend({},e.fn.scrollspy.defaults,n),this.$scrollElement=i.on("scroll.scroll-spy.data-api",r),this.selector=(this.options.target||(s=e(t).attr("href"))&&s.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.$body=e("body"),this.refresh(),this.process()}t.prototype={constructor:t,refresh:function(){var t=this,n;this.offsets=e([]),this.targets=e([]),n=this.$body.find(this.selector).map(function(){var n=e(this),r=n.data("target")||n.attr("href"),i=/^#\w/.test(r)&&e(r);return i&&i.length&&[[i.position().top+(!e.isWindow(t.$scrollElement.get(0))&&t.$scrollElement.scrollTop()),r]]||null}).sort(function(e,t){return e[0]-t[0]
 }).each(function(){t.offsets.push(this[0]),t.targets.push(this[1])})},process:function(){var e=this.$scrollElement.scrollTop()+this.options.offset,t=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,n=t-this.$scrollElement.height(),r=this.offsets,i=this.targets,s=this.activeTarget,o;if(e>=n)return s!=(o=i.last()[0])&&this.activate(o);for(o=r.length;o--;)s!=i[o]&&e>=r[o]&&(!r[o+1]||e<=r[o+1])&&this.activate(i[o])},activate:function(t){var n,r;this.activeTarget=t,e(this.selector).parent(".active").removeClass("active"),r=this.selector+'[data-target="'+t+'"],'+this.selector+'[href="'+t+'"]',n=e(r).parent("li").addClass("active"),n.parent(".dropdown-menu").length&&(n=n.closest("li.dropdown").addClass("active")),n.trigger("activate")}};var n=e.fn.scrollspy;e.fn.scrollspy=function(n){return this.each(function(){var r=e(this),i=r.data("scrollspy"),s=typeof n=="object"&&n;i||r.data("scrollspy",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.scrollspy.Constructor=t,e.fn.scr
 ollspy.defaults={offset:10},e.fn.scrollspy.noConflict=function(){return e.fn.scrollspy=n,this},e(window).on("load",function(){e('[data-spy="scroll"]').each(function(){var t=e(this);t.scrollspy(t.data())})})}(window.jQuery),!function(e){"use strict";var t=function(t){this.element=e(t)};t.prototype={constructor:t,show:function(){var t=this.element,n=t.closest("ul:not(.dropdown-menu)"),r=t.attr("data-target"),i,s,o;r||(r=t.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,""));if(t.parent("li").hasClass("active"))return;i=n.find(".active:last a")[0],o=e.Event("show",{relatedTarget:i}),t.trigger(o);if(o.isDefaultPrevented())return;s=e(r),this.activate(t.parent("li"),n),this.activate(s,s.parent(),function(){t.trigger({type:"shown",relatedTarget:i})})},activate:function(t,n,r){function o(){i.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),t.addClass("active"),s?(t[0].offsetWidth,t.addClass("in")):t.removeClass("fade"),t.parent(".dropdown-menu")&&t.closest("li.dro
 pdown").addClass("active"),r&&r()}var i=n.find("> .active"),s=r&&e.support.transition&&i.hasClass("fade");s?i.one(e.support.transition.end,o):o(),i.removeClass("in")}};var n=e.fn.tab;e.fn.tab=function(n){return this.each(function(){var r=e(this),i=r.data("tab");i||r.data("tab",i=new t(this)),typeof n=="string"&&i[n]()})},e.fn.tab.Constructor=t,e.fn.tab.noConflict=function(){return e.fn.tab=n,this},e(document).on("click.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(t){t.preventDefault(),e(this).tab("show")})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.typeahead.defaults,n),this.matcher=this.options.matcher||this.matcher,this.sorter=this.options.sorter||this.sorter,this.highlighter=this.options.highlighter||this.highlighter,this.updater=this.options.updater||this.updater,this.source=this.options.source,this.$menu=e(this.options.menu),this.shown=!1,this.listen()};t.prototype={constructor:t,select:fu
 nction(){var e=this.$menu.find(".active").attr("data-value");return this.$element.val(this.updater(e)).change(),this.hide()},updater:function(e){return e},show:function(){var t=e.extend({},this.$element.position(),{height:this.$element[0].offsetHeight});return this.$menu.insertAfter(this.$element).css({top:t.top+t.height,left:t.left}).show(),this.shown=!0,this},hide:function(){return this.$menu.hide(),this.shown=!1,this},lookup:function(t){var n;return this.query=this.$element.val(),!this.query||this.query.length<this.options.minLength?this.shown?this.hide():this:(n=e.isFunction(this.source)?this.source(this.query,e.proxy(this.process,this)):this.source,n?this.process(n):this)},process:function(t){var n=this;return t=e.grep(t,function(e){return n.matcher(e)}),t=this.sorter(t),t.length?this.render(t.slice(0,this.options.items)).show():this.shown?this.hide():this},matcher:function(e){return~e.toLowerCase().indexOf(this.query.toLowerCase())},sorter:function(e){var t=[],n=[],r=[],i;whil
 e(i=e.shift())i.toLowerCase().indexOf(this.query.toLowerCase())?~i.indexOf(this.query)?n.push(i):r.push(i):t.push(i);return t.concat(n,r)},highlighter:function(e){var t=this.query.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g,"\\$&");return e.replace(new RegExp("("+t+")","ig"),function(e,t){return"<strong>"+t+"</strong>"})},render:function(t){var n=this;return t=e(t).map(function(t,r){return t=e(n.options.item).attr("data-value",r),t.find("a").html(n.highlighter(r)),t[0]}),t.first().addClass("active"),this.$menu.html(t),this},next:function(t){var n=this.$menu.find(".active").removeClass("active"),r=n.next();r.length||(r=e(this.$menu.find("li")[0])),r.addClass("active")},prev:function(e){var t=this.$menu.find(".active").removeClass("active"),n=t.prev();n.length||(n=this.$menu.find("li").last()),n.addClass("active")},listen:function(){this.$element.on("focus",e.proxy(this.focus,this)).on("blur",e.proxy(this.blur,this)).on("keypress",e.proxy(this.keypress,this)).on("keyup",e.proxy(this.keyup,t
 his)),this.eventSupported("keydown")&&this.$element.on("keydown",e.proxy(this.keydown,this)),this.$menu.on("click",e.proxy(this.click,this)).on("mouseenter","li",e.proxy(this.mouseenter,this)).on("mouseleave","li",e.proxy(this.mouseleave,this))},eventSupported:function(e){var t=e in this.$element;return t||(this.$element.setAttribute(e,"return;"),t=typeof this.$element[e]=="function"),t},move:function(e){if(!this.shown)return;switch(e.keyCode){case 9:case 13:case 27:e.preventDefault();break;case 38:e.preventDefault(),this.prev();break;case 40:e.preventDefault(),this.next()}e.stopPropagation()},keydown:function(t){this.suppressKeyPressRepeat=~e.inArray(t.keyCode,[40,38,9,13,27]),this.move(t)},keypress:function(e){if(this.suppressKeyPressRepeat)return;this.move(e)},keyup:function(e){switch(e.keyCode){case 40:case 38:case 16:case 17:case 18:break;case 9:case 13:if(!this.shown)return;this.select();break;case 27:if(!this.shown)return;this.hide();break;default:this.lookup()}e.stopPropagat
 ion(),e.preventDefault()},focus:function(e){this.focused=!0},blur:function(e){this.focused=!1,!this.mousedover&&this.shown&&this.hide()},click:function(e){e.stopPropagation(),e.preventDefault(),this.select(),this.$element.focus()},mouseenter:function(t){this.mousedover=!0,this.$menu.find(".active").removeClass("active"),e(t.currentTarget).addClass("active")},mouseleave:function(e){this.mousedover=!1,!this.focused&&this.shown&&this.hide()}};var n=e.fn.typeahead;e.fn.typeahead=function(n){return this.each(function(){var r=e(this),i=r.data("typeahead"),s=typeof n=="object"&&n;i||r.data("typeahead",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.typeahead.defaults={source:[],items:8,menu:'<ul class="typeahead dropdown-menu"></ul>',item:'<li><a href="#"></a></li>',minLength:1},e.fn.typeahead.Constructor=t,e.fn.typeahead.noConflict=function(){return e.fn.typeahead=n,this},e(document).on("focus.typeahead.data-api",'[data-provide="typeahead"]',function(t){var n=e(this);if(n.data("typeah
 ead"))return;n.typeahead(n.data())})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=e.extend({},e.fn.affix.defaults,n),this.$window=e(window).on("scroll.affix.data-api",e.proxy(this.checkPosition,this)).on("click.affix.data-api",e.proxy(function(){setTimeout(e.proxy(this.checkPosition,this),1)},this)),this.$element=e(t),this.checkPosition()};t.prototype.checkPosition=function(){if(!this.$element.is(":visible"))return;var t=e(document).height(),n=this.$window.scrollTop(),r=this.$element.offset(),i=this.options.offset,s=i.bottom,o=i.top,u="affix affix-top affix-bottom",a;typeof i!="object"&&(s=o=i),typeof o=="function"&&(o=i.top()),typeof s=="function"&&(s=i.bottom()),a=this.unpin!=null&&n+this.unpin<=r.top?!1:s!=null&&r.top+this.$element.height()>=t-s?"bottom":o!=null&&n<=o?"top":!1;if(this.affixed===a)return;this.affixed=a,this.unpin=a=="bottom"?r.top-n:null,this.$element.removeClass(u).addClass("affix"+(a?"-"+a:""))};var n=e.fn.affix;e.fn.affix=function(
 n){return this.each(function(){var r=e(this),i=r.data("affix"),s=typeof n=="object"&&n;i||r.data("affix",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.affix.Constructor=t,e.fn.affix.defaults={offset:0},e.fn.affix.noConflict=function(){return e.fn.affix=n,this},e(window).on("load",function(){e('[data-spy="affix"]').each(function(){var t=e(this),n=t.data();n.offset=n.offset||{},n.offsetBottom&&(n.offset.bottom=n.offsetBottom),n.offsetTop&&(n.offset.top=n.offsetTop),t.affix(n)})})}(window.jQuery);

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/config/YarnConfig.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/config/YarnConfig.scala b/samza-yarn/src/main/scala/org/apache/samza/config/YarnConfig.scala
new file mode 100644
index 0000000..4779edb
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/config/YarnConfig.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+object YarnConfig {
+  // environment variables
+  val ENV_CONFIG = "STREAMING_CONFIG"
+  val ENV_PARTITION_ID = "PARTITION_ID"
+  val ENV_CONTAINER_NAME = "CONTAINER_NAME"
+
+  // yarn job config
+  val PACKAGE_PATH = "yarn.package.path"
+  val CONTAINER_MAX_MEMORY_MB = "yarn.container.memory.mb"
+  val CONTAINER_MAX_CPU_CORES = "yarn.container.cpu.cores"
+  val CONTAINER_RETRY_COUNT = "yarn.countainer.retry.count"
+  val CONTAINER_RETRY_WINDOW_MS = "yarn.container.retry.window.ms"
+  val TASK_COUNT = "yarn.container.count"
+  val AM_JVM_OPTIONS = "yarn.am.opts"
+  val AM_JMX_ENABLED = "yarn.am.jmx.enabled"
+  val AM_CONTAINER_MAX_MEMORY_MB = "yarn.am.container.memory.mb"
+
+  implicit def Config2Yarn(config: Config) = new YarnConfig(config)
+}
+
+class YarnConfig(config: Config) extends ScalaMapConfig(config) {
+  def getContainerMaxMemoryMb: Option[Int] = getOption(YarnConfig.CONTAINER_MAX_MEMORY_MB) match {
+    case Some(mem) => Some(mem.toInt)
+    case _ => None
+  }
+
+  def getContainerMaxCpuCores: Option[Int] = getOption(YarnConfig.CONTAINER_MAX_CPU_CORES) match {
+    case Some(cpu) => Some(cpu.toInt)
+    case _ => None
+  }
+
+  def getContainerRetryCount: Option[Int] = getOption(YarnConfig.CONTAINER_RETRY_COUNT) match {
+    case Some(count) => Some(count.toInt)
+    case _ => None
+  }
+
+  def getContainerRetryWindowMs: Option[Int] = getOption(YarnConfig.CONTAINER_RETRY_WINDOW_MS) match {
+    case Some(retryWindowMs) => Some(retryWindowMs.toInt)
+    case _ => None
+  }
+
+  def getPackagePath = getOption(YarnConfig.PACKAGE_PATH)
+
+  def getTaskCount: Option[Int] = getOption(YarnConfig.TASK_COUNT) match {
+    case Some(tc) => Some(tc.toInt)
+    case _ => None
+  }
+
+  def getAmOpts = getOption(YarnConfig.AM_JVM_OPTIONS)
+  
+  def getAMContainerMaxMemoryMb: Option[Int] = getOption(YarnConfig.AM_CONTAINER_MAX_MEMORY_MB) match {
+    case Some(mem) => Some(mem.toInt)
+    case _ => None
+  }
+
+  def getJmxServerEnabled = getBoolean(YarnConfig.AM_JMX_ENABLED, true)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
new file mode 100644
index 0000000..b2b529e
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+
+import java.util.Collections
+
+import scala.collection.JavaConversions._
+import scala.collection.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.Resource
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.hadoop.yarn.api.records.ApplicationReport
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+import org.apache.hadoop.yarn.api.ClientRMProtocol
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.hadoop.yarn.util.Records
+
+import org.apache.samza.job.ApplicationStatus
+import org.apache.samza.job.ApplicationStatus.New
+import org.apache.samza.job.ApplicationStatus.Running
+import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
+import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
+
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+
+/**
+ * Client helper class required to submit an application master start script to the resource manager. Also
+ * allows us to forcefully shut-down the application master which in-turn will shut-down the corresponding
+ * container and its processes.
+ */
+class ClientHelper(conf: Configuration) extends Logging {
+  val rpc = YarnRPC.create(conf)
+  val rmAddress = NetUtils.createSocketAddr(conf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS))
+  info("trying to connect to RM %s" format rmAddress)
+  val applicationsManager = rpc.getProxy(classOf[ClientRMProtocol], rmAddress, conf).asInstanceOf[ClientRMProtocol]
+  var appId: Option[ApplicationId] = None
+
+  /**
+   * Generate an application and submit it to the resource manager to start an application master
+   */
+  def submitApplication(packagePath: Path, memoryMb: Int, cpuCore: Int, user: UserGroupInformation, cmds: List[String], env: Option[Map[String, String]], name: Option[String]): Option[ApplicationId] = {
+    val newAppRequest = Records.newRecord(classOf[GetNewApplicationRequest])
+    val newAppResponse = applicationsManager.getNewApplication(newAppRequest)
+    var mem = memoryMb
+    var cpu = cpuCore
+
+    // If we are asking for memory less than the minimum required, bump it
+    if (mem < newAppResponse.getMinimumResourceCapability().getMemory()) {
+      val min = newAppResponse.getMinimumResourceCapability().getMemory()
+      warn("requesting %s megs of memory, which is less than minimum capability of %s, so using minimum" format (mem, min))
+      mem = min
+    }
+
+    // If we are asking for memory more than the max allowed, shout out
+    if (mem > newAppResponse.getMaximumResourceCapability().getMemory()) {
+      throw new SamzaException("You're asking for more memory (%s) than is allowed by YARN: %s" format
+        (mem, newAppResponse.getMaximumResourceCapability().getMemory()))
+    }
+
+    // if we are asking for cpu less than the minimum required, bump it
+    if (cpu < newAppResponse.getMinimumResourceCapability().getVirtualCores()) {
+      val min = newAppResponse.getMinimumResourceCapability.getVirtualCores()
+      warn("requesting %s virtual cores of cpu, which is less than minimum capability of %s, so using minimum" format (cpu, min))
+      cpu = min
+    }
+
+    // If we are asking for cpu more than the max allowed, shout out
+    if (cpu > newAppResponse.getMaximumResourceCapability().getVirtualCores()) {
+      throw new SamzaException("You're asking for more CPU (%s) than is allowed by YARN: %s" format
+        (cpu, newAppResponse.getMaximumResourceCapability().getVirtualCores()))
+    }
+
+    appId = Some(newAppResponse.getApplicationId)
+
+    info("preparing to request resources for app id %s" format appId.get)
+
+    val appCtx = Records.newRecord(classOf[ApplicationSubmissionContext])
+    val containerCtx = Records.newRecord(classOf[ContainerLaunchContext])
+    val resource = Records.newRecord(classOf[Resource])
+    val submitAppRequest = Records.newRecord(classOf[SubmitApplicationRequest])
+    val packageResource = Records.newRecord(classOf[LocalResource])
+
+    name match {
+      case Some(name) => { appCtx.setApplicationName(name) }
+      case None => { appCtx.setApplicationName(appId.toString) }
+    }
+
+    env match {
+      case Some(env) => {
+        containerCtx.setEnvironment(env)
+        info("set environment variables to %s for %s" format (env, appId.get))
+      }
+      case None => None
+    }
+
+    // set the local package so that the containers and app master are provisioned with it
+    val packageUrl = ConverterUtils.getYarnUrlFromPath(packagePath)
+    val fileStatus = packagePath.getFileSystem(conf).getFileStatus(packagePath)
+
+    packageResource.setResource(packageUrl)
+    info("set package url to %s for %s" format (packageUrl, appId.get))
+    packageResource.setSize(fileStatus.getLen)
+    info("set package size to %s for %s" format (fileStatus.getLen, appId.get))
+    packageResource.setTimestamp(fileStatus.getModificationTime)
+    packageResource.setType(LocalResourceType.ARCHIVE)
+    packageResource.setVisibility(LocalResourceVisibility.APPLICATION)
+
+    resource.setMemory(mem)
+    info("set memory request to %s for %s" format (mem, appId.get))
+    resource.setVirtualCores(cpu)
+    info("set cpu core request to %s for %s" format (cpu, appId.get))
+    containerCtx.setResource(resource)
+    containerCtx.setCommands(cmds.toList)
+    info("set command to %s for %s" format (cmds, appId.get))
+    containerCtx.setLocalResources(Collections.singletonMap("__package", packageResource))
+    appCtx.setApplicationId(appId.get)
+    info("set app ID to %s" format (user, appId.get))
+    appCtx.setUser(user.getShortUserName)
+    info("set user to %s for %s" format (user, appId.get))
+    appCtx.setAMContainerSpec(containerCtx)
+    submitAppRequest.setApplicationSubmissionContext(appCtx)
+    info("submitting application request for %s" format appId.get)
+    applicationsManager.submitApplication(submitAppRequest)
+    appId
+  }
+
+  def status(appId: ApplicationId): Option[ApplicationStatus] = {
+    val statusRequest = Records.newRecord(classOf[GetApplicationReportRequest])
+    statusRequest.setApplicationId(appId)
+    val statusResponse = applicationsManager.getApplicationReport(statusRequest)
+    convertState(statusResponse.getApplicationReport)
+  }
+
+  def kill(appId: ApplicationId) {
+    val killRequest = Records.newRecord(classOf[KillApplicationRequest])
+    killRequest.setApplicationId(appId)
+    applicationsManager.forceKillApplication(killRequest)
+  }
+
+  def getApplicationMaster(appId: ApplicationId): Option[ApplicationReport] = {
+    val getAppsReq = Records.newRecord(classOf[GetAllApplicationsRequest])
+    val getAppsRsp = applicationsManager.getAllApplications(getAppsReq)
+
+    getAppsRsp.getApplicationList.filter(appRep => appId.equals(appRep.getApplicationId())).headOption
+  }
+
+  def getApplicationMasters(status: Option[ApplicationStatus]): List[ApplicationReport] = {
+    val getAppsReq = Records.newRecord(classOf[GetAllApplicationsRequest])
+    val getAppsRsp = applicationsManager.getAllApplications(getAppsReq)
+
+    status match {
+      case Some(status) => getAppsRsp.getApplicationList
+        .filter(appRep => status.equals(convertState(appRep).get)).toList
+      case None => getAppsRsp.getApplicationList.toList
+    }
+  }
+
+  private def convertState(appReport: ApplicationReport): Option[ApplicationStatus] = {
+    (appReport.getYarnApplicationState(), appReport.getFinalApplicationStatus()) match {
+      case (YarnApplicationState.FINISHED, FinalApplicationStatus.SUCCEEDED) => Some(SuccessfulFinish)
+      case (YarnApplicationState.KILLED, _) | (YarnApplicationState.FAILED, _) => Some(UnsuccessfulFinish)
+      case (YarnApplicationState.NEW, _) | (YarnApplicationState.SUBMITTED, _) => Some(New)
+      case _ => Some(Running)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMaster.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMaster.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMaster.scala
new file mode 100644
index 0000000..7f830f2
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMaster.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.samza.config.MapConfig
+import org.apache.samza.config.YarnConfig
+import org.apache.samza.config.serializers.JsonConfigSerializer
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.ConverterUtils
+import scala.collection.JavaConversions._
+import org.apache.samza.metrics.{ JmxServer, MetricsRegistryMap }
+import grizzled.slf4j.Logging
+import org.apache.hadoop.yarn.client.AMRMClientImpl
+import org.apache.samza.config.YarnConfig._
+import org.apache.samza.job.yarn.SamzaAppMasterTaskManager._
+
+/**
+ * When YARN executes an application master, it needs a bash command to
+ * execute. For Samza, YARN will execute this main method when starting Samza's
+ * YARN application master.
+ *
+ * <br/><br/>
+ *
+ * The main method gets all of the environment variables (passed by Samza's
+ * YARN client, and YARN itself), and wires up everything to run Samza's
+ * application master.
+ */
+object SamzaAppMaster extends Logging {
+  def main(args: Array[String]) {
+    val containerIdStr = System.getenv(ApplicationConstants.AM_CONTAINER_ID_ENV)
+    info("got container id: %s" format containerIdStr)
+    val containerId = ConverterUtils.toContainerId(containerIdStr)
+    val applicationAttemptId = containerId.getApplicationAttemptId
+    info("got app attempt id: %s" format applicationAttemptId)
+    val nodeHostString = System.getenv(ApplicationConstants.NM_HOST_ENV)
+    info("got node manager host: %s" format nodeHostString)
+    val nodePortString = System.getenv(ApplicationConstants.NM_PORT_ENV)
+    info("got node manager port: %s" format nodePortString)
+    val nodeHttpPortString = System.getenv(ApplicationConstants.NM_HTTP_PORT_ENV)
+    info("got node manager http port: %s" format nodeHttpPortString)
+    val config = new MapConfig(JsonConfigSerializer.fromJson(System.getenv(YarnConfig.ENV_CONFIG)))
+    info("got config: %s" format config)
+    val hConfig = new YarnConfiguration
+    hConfig.set("fs.http.impl", "samza.util.hadoop.HttpFileSystem")
+    val amClient = new AMRMClientImpl(applicationAttemptId)
+    val clientHelper = new ClientHelper(hConfig)
+    val registry = new MetricsRegistryMap
+    val containerMem = config.getContainerMaxMemoryMb.getOrElse(DEFAULT_CONTAINER_MEM)
+    val containerCpu = config.getContainerMaxCpuCores.getOrElse(DEFAULT_CPU_CORES)
+    val jmxServer = if (new YarnConfig(config).getJmxServerEnabled) Some(new JmxServer()) else None
+
+    try {
+      // wire up all of the yarn event listeners
+      val state = new SamzaAppMasterState(-1, containerId, nodeHostString, nodePortString.toInt, nodeHttpPortString.toInt)
+      val service = new SamzaAppMasterService(config, state, registry, clientHelper)
+      val lifecycle = new SamzaAppMasterLifecycle(containerMem, containerCpu, state, amClient, hConfig)
+      val metrics = new SamzaAppMasterMetrics(config, state, registry)
+      val am = new SamzaAppMasterTaskManager({ System.currentTimeMillis }, config, state, amClient, hConfig)
+
+      // run the app master
+      new YarnAppMaster(List(state, service, lifecycle, metrics, am), amClient).run
+    } finally {
+      // jmxServer has to be stopped or will prevent process from exiting.
+      if (jmxServer.isDefined) {
+        jmxServer.get.stop
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterLifecycle.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterLifecycle.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterLifecycle.scala
new file mode 100644
index 0000000..95a6f05
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterLifecycle.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+import org.apache.hadoop.yarn.client.AMRMClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+
+/**
+ * Responsible for managing the lifecycle of the application master. Mostly,
+ * this means registering and unregistering with the RM, and shutting down
+ * when the RM tells us to Reboot.
+ */
+class SamzaAppMasterLifecycle(containerMem: Int, containerCpu: Int, state: SamzaAppMasterState, amClient: AMRMClient, conf: YarnConfiguration) extends YarnAppMasterListener with Logging {
+  var validResourceRequest = true
+  var shutdownMessage: String = null
+
+  override def onInit() {
+    val host = state.nodeHost
+
+    amClient.init(conf);
+    amClient.start
+
+    val response = amClient.registerApplicationMaster(host, state.rpcPort, "%s:%d" format (host, state.trackingPort))
+
+    // validate that the YARN cluster can handle our container resource requirements
+    val maxCapability = response.getMaximumResourceCapability
+    val minCapability = response.getMinimumResourceCapability
+    val maxMem = maxCapability.getMemory
+    val minMem = minCapability.getMemory
+    val maxCpu = maxCapability.getVirtualCores
+    val minCpu = minCapability.getVirtualCores
+
+    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, min-mem: %s, max-cpu: %s, min-cpu: %s" format (maxMem, minMem, maxCpu, minCpu))
+
+    if (containerMem < minMem || containerMem > maxMem || containerCpu < minCpu || containerCpu > maxCpu) {
+      shutdownMessage = "The YARN cluster is unable to run your job due to unsatisfiable resource requirements. You asked for mem: %s, and cpu: %s." format (containerMem, containerCpu)
+      error(shutdownMessage)
+      validResourceRequest = false
+      state.status = FinalApplicationStatus.FAILED
+    }
+  }
+
+  override def onReboot() {
+    throw new SamzaException("Received a reboot signal from the RM, so throwing an exception to reboot the AM.")
+  }
+
+  override def onShutdown() {
+    info("Shutting down.")
+    amClient.unregisterApplicationMaster(state.status, shutdownMessage, null)
+    amClient.stop
+  }
+
+  override def shouldShutdown = !validResourceRequest
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala
new file mode 100644
index 0000000..4f6edfb
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.metrics.JvmMetrics
+import org.apache.samza.config.Config
+import org.apache.samza.task.TaskContext
+import org.apache.samza.Partition
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.MetricsConfig.Config2Metrics
+import org.apache.samza.metrics.MetricsReporterFactory
+import org.apache.samza.util.Util
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+import java.util.Timer
+import java.util.TimerTask
+import org.apache.samza.task.ReadableCollector
+
+object SamzaAppMasterMetrics {
+  val metricsGroup = "samza.yarn.am"
+
+  val sourceName = "ApplicationMaster"
+}
+
+/**
+ * Responsible for wiring up Samza's metrics. Given that Samza has a metric
+ * registry, we might as well use it. This class takes Samza's application
+ * master state, and converts it to metrics.
+ */
+class SamzaAppMasterMetrics(config: Config, state: SamzaAppMasterState, registry: ReadableMetricsRegistry) extends YarnAppMasterListener with Logging {
+  import SamzaAppMasterMetrics._
+
+  val jvm = new JvmMetrics(registry)
+  val mEventLoops = registry.newCounter(metricsGroup, "EventLoops")
+  val reporters = config.getMetricReporterNames.map(reporterName => {
+    val metricsFactoryClassName = config
+      .getMetricsFactoryClass(reporterName)
+      .getOrElse(throw new SamzaException("Metrics reporter %s missing .class config" format reporterName))
+
+    val reporter =
+      Util
+        .getObj[MetricsReporterFactory](metricsFactoryClassName)
+        .getMetricsReporter(reporterName, SamzaAppMasterMetrics.sourceName, config)
+
+    reporter.register(SamzaAppMasterMetrics.sourceName, registry)
+    (reporterName, reporter)
+  }).toMap
+
+  override def onInit() {
+    val mRunningContainers = registry.newGauge(metricsGroup, "RunningContainers", { state.runningTasks.size })
+    val mNeededContainers = registry.newGauge(metricsGroup, "NeededContainers", { state.neededContainers })
+    val mCompletedContainers = registry.newGauge(metricsGroup, "CompletedContainers", { state.completedTasks })
+    val mFailedContainers = registry.newGauge(metricsGroup, "FailedContainers", { state.failedContainers })
+    val mReleasedContainers = registry.newGauge(metricsGroup, "ReleasedContainers", { state.releasedContainers })
+    val mTasks = registry.newGauge(metricsGroup, "TaskCount", { state.taskCount })
+    val mHost = registry.newGauge(metricsGroup, "HttpHost", { state.nodeHost })
+    val mTrackingPort = registry.newGauge(metricsGroup, "HttpPort", { state.trackingPort })
+    val mRpcPort = registry.newGauge(metricsGroup, "RpcPort", { state.rpcPort })
+    val mAppAttemptId = registry.newGauge(metricsGroup, "AppAttemptId", { state.appAttemptId.toString })
+
+    reporters.values.foreach(_.start)
+  }
+
+  override def onEventLoop() {
+    mEventLoops.inc
+  }
+
+  override def onShutdown() {
+    reporters.values.foreach(_.stop)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterService.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterService.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterService.scala
new file mode 100644
index 0000000..ce3fcc3
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterService.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+
+import org.apache.samza.util.Util
+import grizzled.slf4j.Logging
+import org.apache.samza.webapp._
+import org.apache.samza.config.Config
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.SamzaException
+
+/**
+ * Samza's application master runs a very basic HTTP/JSON service to allow
+ * dashboards to check on the status of a job. SamzaAppMasterService starts
+ * up the web service when initialized.
+ */
+class SamzaAppMasterService(config: Config, state: SamzaAppMasterState, registry: ReadableMetricsRegistry, clientHelper: ClientHelper) extends YarnAppMasterListener with Logging {
+  override def onInit() {
+    // try starting the samza AM dashboard. try ten times, just in case we 
+    // pick a port that's already in use.
+    for (i <- 0 until 10) {
+      val rpcPort = Util.randomBetween(10000, 50000)
+      val trackingPort = Util.randomBetween(10000, 50000)
+      info("Starting webapp at rpc %d, tracking port %d" format (rpcPort, trackingPort))
+
+      try {
+        val rpcapp = new WebAppServer("/", rpcPort)
+        rpcapp.addServlet("/*", new ApplicationMasterRestServlet(config, state, registry))
+        rpcapp.start
+
+        val webapp = new WebAppServer("/", trackingPort)
+        webapp.addServlet("/*", new ApplicationMasterWebServlet(config, state))
+        webapp.start
+
+        state.rpcPort = rpcPort
+        state.trackingPort = trackingPort
+        return
+      } catch {
+        case e: Exception => {
+          warn("Unable to start webapp on rpc port %d, tracking port %d .. retrying" format (rpcPort, trackingPort))
+        }
+      }
+    }
+
+    if (state.rpcPort == 0 || state.trackingPort == 0) {
+      throw new SamzaException("Giving up trying to start the webapp, since we keep getting ports that are already in use")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterState.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterState.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterState.scala
new file mode 100644
index 0000000..fa1642b
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterState.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.hadoop.yarn.api.records.ContainerStatus
+import org.apache.hadoop.yarn.api.records.Container
+import org.apache.samza.config.Config
+import grizzled.slf4j.Logging
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+import org.apache.samza.Partition
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.hadoop.yarn.api.records.ContainerId
+
+/**
+ * Samza's application master has state that is usually manipulated based on
+ * responses from YARN's resource manager (via SamzaAppMasterTaskManager). This
+ * class holds the current state of the application master.
+ */
+class SamzaAppMasterState(val taskId: Int, val containerId: ContainerId, val nodeHost: String, val nodePort: Int, val nodeHttpPort: Int) extends YarnAppMasterListener with Logging {
+  // controlled by the AM
+  var completedTasks = 0
+  var neededContainers = 0
+  var failedContainers = 0
+  var releasedContainers = 0
+  var taskCount = 0
+  var unclaimedTasks = Set[Int]()
+  var finishedTasks = Set[Int]()
+  var runningTasks = Map[Int, Container]()
+  var taskPartitions = Map[Int, Set[Partition]]()
+  var status = FinalApplicationStatus.UNDEFINED
+
+  // controlled by the service
+  var trackingPort = 0
+  var rpcPort = 0
+
+  // controlled on startup
+  var appAttemptId = containerId.getApplicationAttemptId
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterTaskManager.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterTaskManager.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterTaskManager.scala
new file mode 100644
index 0000000..1a13ee5
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterTaskManager.scala
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.hadoop.yarn.api.records.ContainerStatus
+import org.apache.hadoop.yarn.api.records.Container
+import org.apache.samza.config.Config
+import org.apache.samza.Partition
+import grizzled.slf4j.Logging
+import org.apache.samza.config.YarnConfig.Config2Yarn
+import org.apache.samza.config.YarnConfig
+import org.apache.samza.job.CommandBuilder
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.fs.Path
+import org.apache.samza.task.TaskContext
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.TaskConfig.Config2Task
+import scala.collection.JavaConversions._
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
+import org.apache.samza.util.Util
+import scala.collection.JavaConversions._
+import org.apache.samza.SamzaException
+import org.apache.hadoop.yarn.client.AMRMClient
+import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.api.records.Priority
+import org.apache.hadoop.yarn.api.records.Resource
+import org.apache.hadoop.yarn.util.Records
+import org.apache.hadoop.security.token.Token
+import org.apache.hadoop.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.api.ContainerManager
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.net.NetUtils
+import java.util.Collections
+import java.security.PrivilegedAction
+import org.apache.samza.job.ShellCommandBuilder
+
+object SamzaAppMasterTaskManager {
+  val DEFAULT_CONTAINER_MEM = 256
+  val DEFAULT_CPU_CORES = 1
+  val DEFAULT_CONTAINER_RETRY_COUNT = 8
+  val DEFAULT_CONTAINER_RETRY_WINDOW_MS = 300000
+
+  def getPartitionsForTask(taskId: Int, taskCount: Int, partitions: Set[Partition]) = {
+    partitions.filter(_.getPartitionId % taskCount == taskId).toSet
+  }
+}
+
+case class TaskFailure(val count: Int, val lastFailure: Long)
+
+/**
+ * Samza's application master is mostly interested in requesting containers to
+ * run Samza jobs. SamzaAppMasterTaskManager is responsible for requesting new
+ * containers, handling failures, and notifying the application master that the
+ * job is done.
+ */
+class SamzaAppMasterTaskManager(clock: () => Long, config: Config, state: SamzaAppMasterState, amClient: AMRMClient, conf: YarnConfiguration) extends YarnAppMasterListener with Logging {
+  import SamzaAppMasterTaskManager._
+
+  state.taskCount = config.getTaskCount match {
+    case Some(count) => count
+    case None =>
+      info("No %s specified. Defaulting to one container." format YarnConfig.TASK_COUNT)
+      1
+  }
+
+  val partitions = Util.getMaxInputStreamPartitions(config)
+  var taskFailures = Map[Int, TaskFailure]()
+  var tooManyFailedContainers = false
+
+  override def shouldShutdown = state.completedTasks == state.taskCount || tooManyFailedContainers
+
+  override def onInit() {
+    state.neededContainers = state.taskCount
+    state.unclaimedTasks = (0 until state.taskCount).toSet
+
+    info("Requesting %s containers" format state.taskCount)
+
+    requestContainers(config.getContainerMaxMemoryMb.getOrElse(DEFAULT_CONTAINER_MEM), config.getContainerMaxCpuCores.getOrElse(DEFAULT_CPU_CORES), state.neededContainers)
+  }
+
+  override def onContainerAllocated(container: Container) {
+    val containerIdStr = ConverterUtils.toString(container.getId)
+
+    info("Got a container from YARN ResourceManager: %s" format container)
+
+    state.unclaimedTasks.headOption match {
+      case Some(taskId) => {
+        info("Got available task id (%d) for container: %s" format (taskId, container))
+        val partitionsForTask = getPartitionsForTask(taskId, state.taskCount, partitions)
+        info("Claimed partitions %s for task ID %s" format (partitionsForTask, taskId))
+        val cmdBuilderClassName = config.getCommandClass.getOrElse(classOf[ShellCommandBuilder].getName)
+        val cmdBuilder = Class.forName(cmdBuilderClassName).newInstance.asInstanceOf[CommandBuilder]
+          .setConfig(config)
+          .setName("SamzaContainer-%s" format taskId)
+          .setPartitions(partitionsForTask)
+          .setTotalPartitions(partitions.size)
+        val command = cmdBuilder.buildCommand
+        info("Task ID %s using command %s" format (taskId, command))
+        val env = cmdBuilder.buildEnvironment.map { case (k, v) => (k, Util.envVarEscape(v)) }
+        info("Task ID %s using env %s" format (taskId, env))
+        val user = UserGroupInformation.getCurrentUser
+        info("Task ID %s using user %s" format (taskId, user))
+        val path = new Path(config.getPackagePath.get)
+        info("Starting task ID %s using package path %s" format (taskId, path))
+
+        startContainer(
+          path,
+          container,
+          user,
+          env.toMap,
+          "export SAMZA_LOG_DIR=%s && ln -sfn %s logs && exec ./__package/%s 1>logs/%s 2>logs/%s" format (ApplicationConstants.LOG_DIR_EXPANSION_VAR, ApplicationConstants.LOG_DIR_EXPANSION_VAR, command, ApplicationConstants.STDOUT, ApplicationConstants.STDERR))
+
+        state.neededContainers -= 1
+        state.runningTasks += taskId -> container
+        state.unclaimedTasks -= taskId
+        state.taskPartitions += taskId -> partitionsForTask
+
+        info("Claimed task ID %s for container %s on node %s (http://%s/node/containerlogs/%s)." format (taskId, containerIdStr, container.getNodeId.getHost, container.getNodeHttpAddress, containerIdStr))
+
+        info("Started task ID %s" format taskId)
+      }
+      case _ => {
+        // there are no more tasks to run, so release the container
+        info("Got an extra container from YARN ResourceManager: %s" format (container))
+
+        amClient.releaseAssignedContainer(container.getId)
+      }
+    }
+  }
+
+  override def onContainerCompleted(containerStatus: ContainerStatus) {
+    val containerIdStr = ConverterUtils.toString(containerStatus.getContainerId)
+    val taskId = state.runningTasks.filter { case (_, container) => container.getId().equals(containerStatus.getContainerId()) }.keys.headOption
+
+    taskId match {
+      case Some(taskId) => {
+        state.runningTasks -= taskId
+        state.taskPartitions -= taskId
+      }
+      case _ => None
+    }
+
+    containerStatus.getExitStatus match {
+      case 0 => {
+        info("Container %s completed successfully." format containerIdStr)
+
+        state.completedTasks += 1
+
+        if (taskId.isDefined) {
+          state.finishedTasks += taskId.get
+          taskFailures -= taskId.get
+        }
+
+        if (state.completedTasks == state.taskCount) {
+          info("Setting job status to SUCCEEDED, since all tasks have been marked as completed.")
+          state.status = FinalApplicationStatus.SUCCEEDED
+        }
+      }
+      case -100 => {
+        info("Got an exit code of -100. This means that container %s was "
+          + "killed by YARN, either due to being released by the application "
+          + "master or being 'lost' due to node failures etc." format containerIdStr)
+
+        state.releasedContainers += 1
+
+        // If this container was assigned some partitions (a taskId), then 
+        // clean up, and request a new container for the tasks. This only 
+        // should happen if the container was 'lost' due to node failure, not 
+        // if the AM released the container.
+        if (taskId.isDefined) {
+          info("Released container %s was assigned task ID %s. Requesting a new container for the task." format (containerIdStr, taskId.get))
+
+          state.neededContainers += 1
+          state.unclaimedTasks += taskId.get
+
+          // request a new container
+          requestContainers(config.getContainerMaxMemoryMb.getOrElse(DEFAULT_CONTAINER_MEM), config.getContainerMaxCpuCores.getOrElse(DEFAULT_CPU_CORES), 1)
+        }
+      }
+      case _ => {
+        info("Container %s failed with exit code %d - %s." format (containerIdStr, containerStatus.getExitStatus, containerStatus.getDiagnostics))
+
+        state.failedContainers += 1
+
+        taskId match {
+          case Some(taskId) =>
+            info("Failed container %s owned task id %s." format (containerIdStr, taskId))
+
+            state.unclaimedTasks += taskId
+            state.neededContainers += 1
+
+            // A container failed for an unknown reason. Let's check to see if 
+            // we need to shutdown the whole app master if too many container 
+            // failures have happened. The rules for failing are that the failure 
+            // count for a task id must be > the configured retry count, and the 
+            // last failure (the one prior to this one) must have happened less 
+            // than retry window ms ago. If retry count is set to 0, the app 
+            // master will fail on any container failure. If the retry count is 
+            // set to a number < 0, a container failure will never trigger an 
+            // app master failure.
+            val retryCount = config.getContainerRetryCount.getOrElse(DEFAULT_CONTAINER_RETRY_COUNT)
+            val retryWindowMs = config.getContainerRetryWindowMs.getOrElse(DEFAULT_CONTAINER_RETRY_WINDOW_MS)
+
+            if (retryCount == 0) {
+              error("Task id %s (%s) failed, and retry count is set to 0, so shutting down the application master, and marking the job as failed."
+                format (taskId, containerIdStr))
+
+              tooManyFailedContainers = true
+            } else if (retryCount > 0) {
+              val (currentFailCount, lastFailureTime) = taskFailures.get(taskId) match {
+                case Some(TaskFailure(count, lastFailure)) => (count + 1, lastFailure)
+                case _ => (1, 0L)
+              }
+
+              if (currentFailCount > retryCount) {
+                val lastFailureMsDiff = clock() - lastFailureTime
+
+                if (lastFailureMsDiff < retryWindowMs) {
+                  error("Task id %s (%s) has failed %s times, with last failure %sms ago. This is greater than retry count of %s and window of %s, so shutting down the application master, and marking the job as failed."
+                    format (taskId, containerIdStr, currentFailCount, lastFailureMsDiff, retryCount, retryWindowMs))
+
+                  // We have too many failures, and we're within the window 
+                  // boundary, so reset shut down the app master.
+                  tooManyFailedContainers = true
+                  state.status = FinalApplicationStatus.FAILED
+                } else {
+                  info("Resetting fail count for task id %s back to 1, since last container failure (%s) for this task id was outside the bounds of the retry window."
+                    format (taskId, containerIdStr))
+
+                  // Reset counter back to 1, since the last failure for this 
+                  // container happened outside the window boundary.
+                  taskFailures += taskId -> TaskFailure(1, clock())
+                }
+              } else {
+                info("Current fail count for task id %s is %s." format (taskId, currentFailCount))
+                taskFailures += taskId -> TaskFailure(currentFailCount, clock())
+              }
+            }
+
+            if (!tooManyFailedContainers) {
+              // Request a new container
+              requestContainers(config.getContainerMaxMemoryMb.getOrElse(DEFAULT_CONTAINER_MEM), config.getContainerMaxCpuCores.getOrElse(DEFAULT_CPU_CORES), 1)
+            }
+          case _ => None
+        }
+      }
+    }
+  }
+
+  protected def startContainer(packagePath: Path, container: Container, ugi: UserGroupInformation, env: Map[String, String], cmds: String*) {
+    info("starting container %s %s %s %s %s" format (packagePath, container, ugi, env, cmds))
+    // connect to container manager (based on similar code in the ContainerLauncher in Hadoop MapReduce)
+    val contToken = container.getContainerToken
+    val address = container.getNodeId.getHost + ":" + container.getNodeId.getPort
+    var user = ugi
+
+    if (UserGroupInformation.isSecurityEnabled) {
+      debug("security is enabled")
+      val hadoopToken = new Token[ContainerTokenIdentifier](contToken.getIdentifier.array, contToken.getPassword.array, new Text(contToken.getKind), new Text(contToken.getService))
+      user = UserGroupInformation.createRemoteUser(address)
+      user.addToken(hadoopToken)
+      debug("changed user to %s" format user)
+    }
+
+    val containerManager = user.doAs(new PrivilegedAction[ContainerManager] {
+      def run(): ContainerManager = {
+        return YarnRPC.create(conf).getProxy(classOf[ContainerManager], NetUtils.createSocketAddr(address), conf).asInstanceOf[ContainerManager]
+      }
+    })
+
+    // set the local package so that the containers and app master are provisioned with it
+    val packageResource = Records.newRecord(classOf[LocalResource])
+    val packageUrl = ConverterUtils.getYarnUrlFromPath(packagePath)
+    val fileStatus = packagePath.getFileSystem(conf).getFileStatus(packagePath)
+
+    packageResource.setResource(packageUrl)
+    packageResource.setSize(fileStatus.getLen)
+    packageResource.setTimestamp(fileStatus.getModificationTime)
+    packageResource.setType(LocalResourceType.ARCHIVE)
+    packageResource.setVisibility(LocalResourceVisibility.APPLICATION)
+
+    // start the container
+    val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+    ctx.setEnvironment(env)
+    ctx.setContainerId(container.getId())
+    ctx.setResource(container.getResource())
+    ctx.setUser(user.getShortUserName())
+    ctx.setCommands(cmds.toList)
+    ctx.setLocalResources(Collections.singletonMap("__package", packageResource))
+
+    debug("setting package to %s" format packageResource)
+    debug("setting context to %s" format ctx)
+
+    val startContainerRequest = Records.newRecord(classOf[StartContainerRequest])
+    startContainerRequest.setContainerLaunchContext(ctx)
+    containerManager.startContainer(startContainerRequest)
+  }
+
+  protected def requestContainers(memMb: Int, cpuCores: Int, containers: Int) {
+    info("Requesting %d container(s) with %dmb of memory" format (containers, memMb))
+    val capability = Records.newRecord(classOf[Resource])
+    val priority = Records.newRecord(classOf[Priority])
+    priority.setPriority(0)
+    capability.setMemory(memMb)
+    capability.setVirtualCores(cpuCores)
+    amClient.addContainerRequest(new ContainerRequest(capability, null, null, priority, containers))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMaster.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMaster.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMaster.scala
new file mode 100644
index 0000000..14e3865
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMaster.scala
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+
+import scala.collection.JavaConversions._
+import grizzled.slf4j.Logging
+import org.apache.hadoop.yarn.client.AMRMClient
+
+/**
+ * YARN's API is somewhat clunky. Most implementations just sit in a loop, and
+ * poll the resource manager every N seconds (see the distributed shell
+ * example). To make life slightly better, Samza separates the polling logic
+ * from the application master logic, and we convert synchronous polling calls
+ * to callbacks, which are more intuitive when dealing with event based
+ * paradigms like YARN.
+ *
+ * <br/><br/>
+ *
+ * SamzaAppMaster uses this class to wire up all of Samza's application master
+ * listeners.
+ */
+class YarnAppMaster(pollIntervalMs: Long, listeners: List[YarnAppMasterListener], amClient: AMRMClient) extends Logging {
+  var isShutdown = false
+
+  def this(listeners: List[YarnAppMasterListener], amClient: AMRMClient) = this(1000, listeners, amClient)
+
+  def run {
+    try {
+      listeners.foreach(_.onInit)
+
+      while (!isShutdown && !listeners.map(_.shouldShutdown).reduceLeft(_ || _)) {
+        val response = amClient.allocate(0).getAMResponse
+
+        if (response.getReboot) {
+          listeners.foreach(_.onReboot)
+        }
+
+        listeners.foreach(_.onEventLoop)
+        response.getCompletedContainersStatuses.foreach(containerStatus => listeners.foreach(_.onContainerCompleted(containerStatus)))
+        response.getAllocatedContainers.foreach(container => listeners.foreach(_.onContainerAllocated(container)))
+
+        try {
+          Thread.sleep(pollIntervalMs)
+        } catch {
+          case e: InterruptedException => {
+            isShutdown = true
+            info("got interrupt in app master thread, so shutting down")
+          }
+        }
+      }
+    } finally {
+      listeners.foreach(listener => try {
+        listener.onShutdown
+      } catch {
+        case e: Throwable => warn("Listener %s failed to shutdown." format listener, e)
+      })
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala
new file mode 100644
index 0000000..1353e86
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.hadoop.yarn.api.records.Container
+import org.apache.hadoop.yarn.api.records.ContainerStatus
+
+/**
+ * Classes that wish to listen to callback events from YarnAppMaster must
+ * implement this trait.
+ */
+trait YarnAppMasterListener {
+  /**
+   * If true, YarnAppMaster will cease to poll the RM, and call onShutdown for
+   * all listeners.
+   */
+  def shouldShutdown: Boolean = false
+
+  /**
+   * Invoked by YarnAppMaster once per listener, before entering the RM polling
+   * event loop.
+   */
+  def onInit() {}
+
+  /**
+   * Invoked whenever the RM responds with a reboot request. Usually, reboots
+   * are triggered by the YARN RM when its state gets out of sync with the
+   * application master (usually the result of restarting the RM).
+   * YarnAppMaster does not actually restart anything. It is up to one or more
+   * listeners to trigger a failure, or shutdown.
+   */
+  def onReboot() {}
+
+  /**
+   * Signifies that the YarnAppMaster has exited the RM polling event loop, and
+   * is about to exit.
+   */
+  def onShutdown() {}
+
+  /**
+   * Whenever the RM allocates a container for the application master, this
+   * callback is invoked (once per container).
+   */
+  def onContainerAllocated(container: Container) {}
+
+  /**
+   * Whenever a container completes (either failure, or success), this callback
+   * will be invoked.
+   */
+  def onContainerCompleted(containerStatus: ContainerStatus) {}
+
+  /**
+   * Invoked by YarnAppMaster once per listener, every time it loops around to
+   * poll the RM again.
+   */
+  def onEventLoop() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala
new file mode 100644
index 0000000..bde38e1
--- /dev/null
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.yarn
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.samza.config.Config
+import org.apache.samza.util.Util
+import org.apache.samza.job.ApplicationStatus
+import org.apache.samza.job.ApplicationStatus.Running
+import org.apache.samza.job.StreamJob
+import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
+import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
+import org.apache.samza.config.serializers.JsonConfigSerializer
+import org.apache.samza.config.YarnConfig.Config2Yarn
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.config.YarnConfig
+import org.apache.samza.config.ShellCommandConfig
+import org.apache.samza.SamzaException
+
+/**
+ * Starts the application manager
+ */
+class YarnJob(config: Config, hadoopConfig: Configuration) extends StreamJob {
+  val client = new ClientHelper(hadoopConfig)
+  var appId: Option[ApplicationId] = None
+
+  def submit: YarnJob = {
+    appId = client.submitApplication(
+      new Path(config.getPackagePath.getOrElse(throw new SamzaException("No YARN package path defined in config."))),
+      config.getAMContainerMaxMemoryMb.getOrElse(512),
+      1,
+      UserGroupInformation.getCurrentUser,
+      List(
+        "export SAMZA_LOG_DIR=%s && ln -sfn %s logs && exec ./__package/bin/run-am.sh 1>logs/%s 2>logs/%s"
+          format (ApplicationConstants.LOG_DIR_EXPANSION_VAR, ApplicationConstants.LOG_DIR_EXPANSION_VAR, ApplicationConstants.STDOUT, ApplicationConstants.STDERR)),
+      Some(Map(
+        YarnConfig.ENV_CONFIG -> Util.envVarEscape(JsonConfigSerializer.toJson(config)),
+        ShellCommandConfig.ENV_SAMZA_OPTS -> Util.envVarEscape(config.getAmOpts.getOrElse("")))),
+      Some("%s_%s" format (config.getName.get, config.getJobId.getOrElse(1))))
+
+    this
+  }
+
+  def waitForFinish(timeoutMs: Long): ApplicationStatus = {
+    val startTimeMs = System.currentTimeMillis()
+
+    while (System.currentTimeMillis() - startTimeMs < timeoutMs) {
+      Option(getStatus) match {
+        case Some(s) => if (SuccessfulFinish.equals(s) || UnsuccessfulFinish.equals(s)) return s
+        case None => null
+      }
+
+      Thread.sleep(1000)
+    }
+
+    Running
+  }
+
+  def waitForStatus(status: ApplicationStatus, timeoutMs: Long): ApplicationStatus = {
+    val startTimeMs = System.currentTimeMillis()
+
+    while (System.currentTimeMillis() - startTimeMs < timeoutMs) {
+      Option(getStatus) match {
+        case Some(s) => if (status.equals(s)) return status
+        case None => null
+      }
+
+      Thread.sleep(1000)
+    }
+
+    Running
+  }
+
+  def getStatus: ApplicationStatus = {
+    appId match {
+      case Some(appId) => client.status(appId).getOrElse(null)
+      case None => null
+    }
+  }
+
+  def kill: YarnJob = {
+    appId match {
+      case Some(appId) => client.kill(appId)
+      case None => None
+    }
+    this
+  }
+}


[05/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/checkpoint/TestKafkaCheckpointManager.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/TestKafkaCheckpointManager.scala
new file mode 100644
index 0000000..baf4695
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/TestKafkaCheckpointManager.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint.kafka
+
+import org.I0Itec.zkclient.ZkClient
+import org.junit.Assert._
+import org.junit.AfterClass
+import org.junit.BeforeClass
+import org.junit.Test
+import kafka.producer.Producer
+import kafka.producer.ProducerConfig
+import kafka.server.KafkaConfig
+import kafka.server.KafkaServer
+import kafka.utils.TestUtils
+import kafka.utils.TestZKUtils
+import kafka.utils.Utils
+import kafka.zk.EmbeddedZookeeper
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.Partition
+import scala.collection._
+import scala.collection.JavaConversions._
+import org.apache.samza.util.{ ClientUtilTopicMetadataStore, TopicMetadataStore }
+import org.apache.samza.config.MapConfig
+import org.apache.samza.checkpoint.Checkpoint
+import org.apache.samza.system.SystemStream
+import kafka.utils.ZKStringSerializer
+
+object TestKafkaCheckpointManager {
+  val zkConnect: String = TestZKUtils.zookeeperConnect
+  var zkClient: ZkClient = null
+  val zkConnectionTimeout = 6000
+  val zkSessionTimeout = 6000
+
+  val brokerId1 = 0
+  val brokerId2 = 1
+  val brokerId3 = 2
+  val ports = TestUtils.choosePorts(3)
+  val (port1, port2, port3) = (ports(0), ports(1), ports(2))
+
+  val props1 = TestUtils.createBrokerConfig(brokerId1, port1)
+  val config1 = new KafkaConfig(props1) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+  }
+  val props2 = TestUtils.createBrokerConfig(brokerId2, port2)
+  val config2 = new KafkaConfig(props2) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+  }
+  val props3 = TestUtils.createBrokerConfig(brokerId3, port3)
+  val config3 = new KafkaConfig(props3) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+  }
+
+  val config = new java.util.Properties()
+  val brokers = "localhost:%d,localhost:%d,localhost:%d" format (port1, port2, port3)
+  config.put("metadata.broker.list", brokers)
+  config.put("producer.type", "sync")
+  config.put("request.required.acks", "-1")
+  val producerConfig = new ProducerConfig(config)
+  val partition = new Partition(0)
+  val cp1 = new Checkpoint(Map(new SystemStream("kafka", "topic") -> "123"))
+  val cp2 = new Checkpoint(Map(new SystemStream("kafka", "topic") -> "12345"))
+  var zookeeper: EmbeddedZookeeper = null
+  var server1: KafkaServer = null
+  var server2: KafkaServer = null
+  var server3: KafkaServer = null
+  var metadataStore: TopicMetadataStore = null
+
+  @BeforeClass
+  def beforeSetupServers {
+    zookeeper = new EmbeddedZookeeper(zkConnect)
+    server1 = TestUtils.createServer(config1)
+    server2 = TestUtils.createServer(config2)
+    server3 = TestUtils.createServer(config3)
+    metadataStore = new ClientUtilTopicMetadataStore(brokers, "some-job-name")
+  }
+
+  @AfterClass
+  def afterCleanLogDirs {
+    server1.shutdown
+    server1.awaitShutdown()
+    server2.shutdown
+    server2.awaitShutdown()
+    server3.shutdown
+    server3.awaitShutdown()
+    Utils.rm(server1.config.logDirs)
+    Utils.rm(server2.config.logDirs)
+    Utils.rm(server3.config.logDirs)
+  }
+}
+
+class TestKafkaCheckpointManager {
+  import TestKafkaCheckpointManager._
+
+  @Test
+  def testCheckpointShouldBeNullIfStateTopicDoesNotExistShouldBeCreatedOnWriteAndShouldBeReadableAfterWrite {
+    val kcm = getKafkaCheckpointManager
+    kcm.register(partition)
+    kcm.start
+    var readCp = kcm.readLastCheckpoint(partition)
+    // read before topic exists should result in a null checkpoint
+    assert(readCp == null)
+    // create topic the first time around
+    kcm.writeCheckpoint(partition, cp1)
+    readCp = kcm.readLastCheckpoint(partition)
+    assert(cp1.equals(readCp))
+    // should get an exception if partition doesn't exist
+    try {
+      readCp = kcm.readLastCheckpoint(new Partition(1))
+      fail("Expected a SamzaException, since only one partition (partition 0) should exist.")
+    } catch {
+      case e: Exception => None // expected
+    }
+    // writing a second message should work, too
+    kcm.writeCheckpoint(partition, cp2)
+    readCp = kcm.readLastCheckpoint(partition)
+    assert(cp2.equals(readCp))
+    kcm.stop
+  }
+
+  private def getKafkaCheckpointManager = new KafkaCheckpointManager(
+    clientId = "some-client-id",
+    stateTopic = "state-topic",
+    systemName = "kafka",
+    totalPartitions = 1,
+    replicationFactor = 3,
+    socketTimeout = 30000,
+    bufferSize = 64 * 1024,
+    fetchSize = 300 * 1024,
+    metadataStore = metadataStore,
+    connectProducer = () => new Producer[Partition, Array[Byte]](producerConfig),
+    connectZk = () => new ZkClient(zkConnect, 6000, 6000, ZKStringSerializer))
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaSerdeConfig.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaSerdeConfig.scala b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaSerdeConfig.scala
new file mode 100644
index 0000000..2e7459e
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaSerdeConfig.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+import org.junit.Assert._
+import org.junit.Test
+import scala.collection.JavaConversions._
+import org.apache.samza.config.KafkaSerdeConfig.Config2KafkaSerde
+
+class TestKafkaSerdeConfig {
+  val MAGIC_VAL = "1000"
+
+  val paramsToTest = List(
+    "serializers.registry.test.encoder", "serializers.registry.test.decoder")
+
+  val config = new MapConfig(asJavaMap(paramsToTest.map { m => (m, MAGIC_VAL) } toMap))
+
+  @Test
+  def testKafkaConfigurationIsBackwardsCompatible {
+    assert(config.getKafkaEncoder("test").getOrElse("").equals(MAGIC_VAL))
+    assert(config.getKafkaDecoder("test").getOrElse("").equals(MAGIC_VAL))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/config/TestRegExTopicGenerator.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/config/TestRegExTopicGenerator.scala b/samza-kafka/src/test/scala/org/apache/samza/config/TestRegExTopicGenerator.scala
new file mode 100644
index 0000000..9122848
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/config/TestRegExTopicGenerator.scala
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import org.junit.Test
+import collection.JavaConversions._
+import org.junit.Assert._
+import KafkaConfig._
+import org.apache.samza.SamzaException
+
+class TestRegExTopicGenerator {
+
+  private def REWRITER_NAME = "test"
+  private def getRegexConfigKey = REGEX_RESOLVED_STREAMS format REWRITER_NAME
+  private def getRegexConfigSystem = REGEX_RESOLVED_SYSTEM format REWRITER_NAME
+  private def getRegexConfigInherited = REGEX_INHERITED_CONFIG format REWRITER_NAME
+
+  @Test
+  def configsAreRewrittenCorrectly = {
+    val unrelated = "unrelated.key.howdy" -> "should.survive"
+    val map = Map(
+      getRegexConfigKey -> ".*cat",
+      getRegexConfigSystem -> "test",
+      getRegexConfigInherited + ".ford" -> "mustang",
+      getRegexConfigInherited + ".alfa.romeo" -> "spider",
+      getRegexConfigInherited + ".b.triumph" -> "spitfire",
+      unrelated)
+
+    val config = new MapConfig(map)
+
+    // Don't actually talk to ZooKeeper
+    val rewriter = new RegexTopicGenerator() {
+      override def getTopicsFromZK(rewriterName: String, config: Config): Seq[String] = List("catdog", "dogtired", "cow", "scaredycat", "Homer", "crazycat")
+    }
+
+    val rewritten = rewriter.rewrite(REWRITER_NAME, config)
+
+    val expected = Map(
+      "task.inputs" -> "test.scaredycat,test.crazycat",
+      "systems.test.streams.scaredycat.ford" -> "mustang",
+      "systems.test.streams.scaredycat.alfa.romeo" -> "spider",
+      "systems.test.streams.scaredycat.b.triumph" -> "spitfire",
+      "systems.test.streams.crazycat.ford" -> "mustang",
+      "systems.test.streams.crazycat.alfa.romeo" -> "spider",
+      "systems.test.streams.crazycat.b.triumph" -> "spitfire",
+      unrelated)
+
+    expected.foreach(e => assertEquals(e._2, rewritten.get(e._1))) // Compiler bug in 2.8 requires this dumb syntax
+    val inputStreams = rewritten.get(TaskConfig.INPUT_STREAMS).split(",").toSet
+    assertEquals(2, inputStreams.size)
+    assertEquals(Set("test.crazycat", "test.scaredycat"), inputStreams)
+  }
+
+  @Test
+  def emptyInputStreamsWorks = {
+    // input.streams is required but appears as the empty string, which has been problematic.
+    val map = Map(
+      TaskConfig.INPUT_STREAMS -> "",
+      getRegexConfigKey -> "yo.*",
+      getRegexConfigSystem -> "test",
+      getRegexConfigInherited + ".config.zorp" -> "morp")
+
+    val rewriter = new RegexTopicGenerator() {
+      override def getTopicsFromZK(rewriterName: String, config: Config): Seq[String] = List("yoyoyo")
+    }
+
+    val config = rewriter.rewrite(REWRITER_NAME, new MapConfig(map))
+    assertEquals("test.yoyoyo", config.get(TaskConfig.INPUT_STREAMS))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/serializers/TestKafkaSerde.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/serializers/TestKafkaSerde.scala b/samza-kafka/src/test/scala/org/apache/samza/serializers/TestKafkaSerde.scala
new file mode 100644
index 0000000..f6fa6bb
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/serializers/TestKafkaSerde.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import org.junit.Assert._
+import org.junit.Test
+import kafka.serializer.StringEncoder
+import kafka.serializer.StringDecoder
+
+class TestKafkaSerde {
+  @Test
+  def testKafkaSerdeShouldWrapEncoderAndDecoders {
+    val serde = new KafkaSerde(new StringEncoder, new StringDecoder)
+    serde.fromBytes(serde.toBytes("test")).equals("test")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala
new file mode 100644
index 0000000..15510ae
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestBrokerProxy.scala
@@ -0,0 +1,178 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.junit._
+import org.junit.Assert._
+import org.apache.samza.config.MapConfig
+import org.mockito.Mockito
+import org.apache.samza.metrics._
+import scala.collection.JavaConversions._
+import kafka.consumer.SimpleConsumer
+import org.mockito.Mockito._
+import org.mockito.Matchers._
+import kafka.api._
+import kafka.message.{ Message, MessageSet, MessageAndOffset }
+import kafka.common.TopicAndPartition
+import kafka.api.PartitionOffsetsResponse
+import java.nio.ByteBuffer
+import org.apache.samza.SamzaException
+import kafka.message.ByteBufferMessageSet
+
+class TestBrokerProxy {
+  def getMockBrokerProxy() = {
+    val sink = new MessageSink {
+      val receivedMessages = new scala.collection.mutable.ListBuffer[(TopicAndPartition, MessageAndOffset, Boolean)]()
+      def abdicate(tp: TopicAndPartition, lastOffset: Long) {}
+
+      def addMessage(tp: TopicAndPartition, msg: MessageAndOffset, highWatermark: Long) { receivedMessages.add((tp, msg, msg.offset.equals(highWatermark))) }
+
+      def setIsAtHighWatermark(tp: TopicAndPartition, isAtHighWatermark: Boolean) {
+      }
+    }
+
+    val config = new MapConfig(Map[String, String]("job.name" -> "Jobby McJob",
+      "systems.daSystem.Redbird.consumer.auto.offset.reset" -> "largest"))
+    val metricsRegistry = {
+      val registry = Mockito.mock(classOf[MetricsRegistry])
+      val gauge = Mockito.mock(classOf[Gauge[Long]])
+      when(gauge.getValue()).thenReturn(0l)
+      when(registry.newGauge[Long](anyString(), anyString(), anyLong())).thenReturn(gauge)
+
+      val counter = Mockito.mock(classOf[Counter])
+      when(registry.newCounter(anyString(), anyString())).thenReturn(counter)
+      registry
+    }
+
+    val tp = new TopicAndPartition("Redbird", 2012)
+    val tpMetrics = new TopicAndPartitionMetrics(metricsRegistry)
+
+    tpMetrics.addNewTopicAndPartition(tp)
+
+    val bp = new BrokerProxy(
+      "host",
+      2222,
+      "daSystem",
+      "daClientId",
+      metricsRegistry,
+      tpMetrics,
+      offsetGetter = new GetOffset("fail", Map("Redbird" -> "largest"))) {
+
+      val messageSink: MessageSink = sink
+
+      override val sleepMSWhileNoTopicPartitions = 100 // Speed up for test
+      var alreadyCreatedConsumer = false
+      // Scala traits and Mockito mocks don't mix, unfortunately.
+      override def createSimpleConsumer() = {
+        if (alreadyCreatedConsumer) {
+          System.err.println("Should only be creating one consumer in this test!")
+          throw new InterruptedException("Should only be creating one consumer in this test!")
+        }
+        alreadyCreatedConsumer = true
+
+        new SimpleConsumer("a", 1, 2, 3, "b") with DefaultFetch {
+          val fetchSize: Int = 42
+
+          val sc = Mockito.mock(classOf[SimpleConsumer])
+          val mockOffsetResponse = {
+            val offsetResponse = Mockito.mock(classOf[OffsetResponse])
+            val partitionOffsetResponse = {
+              val por = Mockito.mock(classOf[PartitionOffsetsResponse])
+              when(por.offsets).thenReturn(List(1l).toSeq)
+              por
+            }
+
+            val map = scala.Predef.Map[TopicAndPartition, PartitionOffsetsResponse](tp -> partitionOffsetResponse)
+            when(offsetResponse.partitionErrorAndOffsets).thenReturn(map)
+            offsetResponse
+          }
+
+          when(sc.getOffsetsBefore(any(classOf[OffsetRequest]))).thenReturn(mockOffsetResponse)
+
+          val fetchResponse = {
+            val fetchResponse = Mockito.mock(classOf[FetchResponse])
+
+            val messageSet = {
+              val messageSet = Mockito.mock(classOf[ByteBufferMessageSet])
+
+              def getMessage() = new Message(Mockito.mock(classOf[ByteBuffer]))
+              val messages = List(new MessageAndOffset(getMessage, 42), new MessageAndOffset(getMessage, 84))
+
+              when(messageSet.iterator).thenReturn(messages.iterator)
+              when(messageSet.head).thenReturn(messages.head)
+              messageSet
+            }
+
+            val fetchResponsePartitionData = FetchResponsePartitionData(0, 500, messageSet)
+            val map = scala.Predef.Map[TopicAndPartition, FetchResponsePartitionData](tp -> fetchResponsePartitionData)
+
+            when(fetchResponse.data).thenReturn(map)
+            when(fetchResponse.messageSet(any(classOf[String]), any(classOf[Int]))).thenReturn(messageSet)
+            fetchResponse
+          }
+          when(sc.fetch(any(classOf[FetchRequest]))).thenReturn(fetchResponse)
+
+          override def close() = sc.close()
+
+          override def send(request: TopicMetadataRequest): TopicMetadataResponse = sc.send(request)
+
+          override def fetch(request: FetchRequest): FetchResponse = sc.fetch(request)
+
+          override def getOffsetsBefore(request: OffsetRequest): OffsetResponse = sc.getOffsetsBefore(request)
+
+          override def commitOffsets(request: OffsetCommitRequest): OffsetCommitResponse = sc.commitOffsets(request)
+
+          override def fetchOffsets(request: OffsetFetchRequest): OffsetFetchResponse = sc.fetchOffsets(request)
+
+          override def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = sc.earliestOrLatestOffset(topicAndPartition, earliestOrLatest, consumerId)
+        }
+      }
+
+    }
+
+    (bp, tp, sink)
+  }
+
+  @Test def brokerProxyRetrievesMessagesCorrectly() = {
+    val (bp, tp, sink) = getMockBrokerProxy()
+
+    bp.start
+    bp.addTopicPartition(tp, "0")
+    Thread.sleep(1000)
+    assertEquals(2, sink.receivedMessages.size)
+    assertEquals(42, sink.receivedMessages.get(0)._2.offset)
+    assertEquals(84, sink.receivedMessages.get(1)._2.offset)
+  }
+
+  @Test def brokerProxyThrowsExceptionOnDuplicateTopicPartitions() = {
+    val (bp, tp, _) = getMockBrokerProxy()
+    bp.start
+    bp.addTopicPartition(tp, "0")
+
+    try {
+      bp.addTopicPartition(tp, "1")
+      fail("Should have thrown an exception")
+    } catch {
+      case se: SamzaException => assertEquals(se.getMessage, "Already consuming TopicPartition [Redbird,2012]")
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemFactory.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemFactory.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemFactory.scala
new file mode 100644
index 0000000..2a74ea5
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemFactory.scala
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import scala.collection.JavaConversions._
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.SamzaException
+import org.apache.samza.config.MapConfig
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.system.SystemStream
+
+class TestKafkaSystemFactory {
+  @Test
+  def testFailWhenNoSerdeDefined {
+    val producerFactory = new KafkaSystemFactory
+    try {
+      producerFactory.getProducer(
+        "test",
+        new MapConfig(Map[String, String]()),
+        new MetricsRegistryMap)
+      fail("Expected to get a Samza exception.")
+    } catch {
+      case e: SamzaException => None // expected
+      case _ => fail("Expected to get a Samza exception.")
+    }
+  }
+
+  @Test
+  def testFailWhenSerdeIsInvalid {
+    val producerFactory = new KafkaSystemFactory
+    val config = new MapConfig(Map[String, String](
+      "streams.test.serde" -> "failme"))
+    try {
+      producerFactory.getProducer(
+        "test",
+        config,
+        new MetricsRegistryMap)
+      fail("Expected to get a Samza exception.")
+    } catch {
+      case e: SamzaException => None // expected
+      case _ => fail("Expected to get a Samza exception.")
+    }
+  }
+
+  @Test
+  def testHappyPath {
+    val producerFactory = new KafkaSystemFactory
+    val config = new MapConfig(Map[String, String](
+      "job.name" -> "test",
+      "systems.test.producer.metadata.broker.list" -> "",
+      "systems.test.samza.key.serde" -> "json",
+      "systems.test.samza.msg.serde" -> "json",
+      "serializers.registry.json.class" -> "samza.serializers.JsonSerdeFactory"))
+    var producer = producerFactory.getProducer(
+      "test",
+      config,
+      new MetricsRegistryMap)
+    assertTrue(producer != null)
+    assertTrue(producer.isInstanceOf[KafkaSystemProducer])
+    producer = producerFactory.getProducer(
+      "test",
+      config,
+      new MetricsRegistryMap)
+    assertTrue(producer != null)
+    assertTrue(producer.isInstanceOf[KafkaSystemProducer])
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala
new file mode 100644
index 0000000..25ed013
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemProducer.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+import org.junit.Assert._
+import org.junit.Test
+import kafka.producer.Producer
+import kafka.producer.async.DefaultEventHandler
+import kafka.producer.ProducerPool
+import kafka.serializer.Encoder
+import java.nio.ByteBuffer
+import kafka.producer.ProducerConfig
+import java.util.Properties
+import scala.collection.JavaConversions._
+import kafka.producer.KeyedMessage
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.system.OutgoingMessageEnvelope
+import org.apache.samza.system.SystemStream
+
+class TestKafkaSystemProducer {
+
+  val someMessage = new OutgoingMessageEnvelope(new SystemStream("test", "test"), "test")
+
+  def getProps = {
+    val props = new Properties
+    props.put("broker.list", "")
+    props.put("metadata.broker.list", "")
+    props
+  }
+
+  @Test
+  def testKafkaProducer {
+    val props = getProps
+    @volatile var msgsSent = new CountDownLatch(1)
+
+    val producer = new KafkaSystemProducer("test", 1, 100, new MetricsRegistryMap, () => {
+      new Producer[Object, Object](new ProducerConfig(props)) {
+        override def send(messages: KeyedMessage[Object, Object]*) {
+          msgsSent.countDown
+        }
+      }
+    })
+
+    producer.register("test")
+    producer.start
+    producer.send("test", someMessage)
+    producer.stop
+    msgsSent.await(120, TimeUnit.SECONDS)
+    assertEquals(0, msgsSent.getCount)
+  }
+
+  @Test
+  def testKafkaProducerBatch {
+    val props = getProps
+    @volatile var msgsSent = 0
+
+    val producer = new KafkaSystemProducer("test", 2, 100, new MetricsRegistryMap, () => {
+      new Producer[Object, Object](new ProducerConfig(props)) {
+        override def send(messages: KeyedMessage[Object, Object]*) {
+          msgsSent += 1
+        }
+      }
+    })
+
+    // second message should trigger the count down
+    producer.register("test")
+    producer.start
+    producer.send("test", someMessage)
+    assertEquals(0, msgsSent)
+    producer.send("test", someMessage)
+    assertEquals(1, msgsSent)
+    producer.stop
+  }
+
+  @Test
+  def testKafkaProducerCommit {
+    val props = getProps
+    val msgs = scala.collection.mutable.ListBuffer[String]()
+    val msg1 = new OutgoingMessageEnvelope(new SystemStream("test", "test"), "a")
+    val msg2 = new OutgoingMessageEnvelope(new SystemStream("test", "test"), "b")
+
+    val producer = new KafkaSystemProducer("test", 3, 100, new MetricsRegistryMap, () => {
+      new Producer[Object, Object](new ProducerConfig(props)) {
+        override def send(messages: KeyedMessage[Object, Object]*) {
+          msgs ++= messages.map(_.message.asInstanceOf[String])
+        }
+      }
+    })
+
+    // commit should trigger the count down
+    producer.register("test")
+    producer.start
+    producer.send("test", msg1)
+    producer.send("test", msg2)
+    assertEquals(0, msgs.size)
+    producer.commit("test")
+    assertEquals(2, msgs.size)
+    assertEquals("a", msgs(0))
+    assertEquals("b", msgs(1))
+    producer.stop
+  }
+
+  @Test
+  def testKafkaSyncProducerExceptions {
+    var msgsSent = 0
+    val props = new Properties
+    val out = new OutgoingMessageEnvelope(new SystemStream("test", "test"), "a")
+    props.put("metadata.broker.list", "")
+    props.put("broker.list", "")
+    props.put("producer.type", "sync")
+
+    var failCount = 0
+    val producer = new KafkaSystemProducer("test", 1, 100, new MetricsRegistryMap, () => {
+      failCount += 1
+      if (failCount <= 5) {
+        throw new RuntimeException("Pretend to fail in factory")
+      }
+      new Producer[Object, Object](new ProducerConfig(props)) {
+        override def send(messages: KeyedMessage[Object, Object]*) {
+          assertNotNull(messages)
+          assertEquals(1, messages.length)
+          assertEquals(messages(0).message, "a")
+          msgsSent += 1
+          if (msgsSent <= 5) {
+            throw new RuntimeException("Pretend to fail in send")
+          }
+        }
+      }
+    })
+
+    producer.register("test")
+    producer.start
+    producer.send("test", out)
+    producer.stop
+    assertEquals(6, msgsSent)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala
new file mode 100644
index 0000000..3c9c8d6
--- /dev/null
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestTopicMetadataCache.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+import org.junit.Test
+import kafka.api.TopicMetadata
+import org.apache.samza.util.TopicMetadataStore
+import org.I0Itec.zkclient.ZkClient
+import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
+import java.util.concurrent.CountDownLatch
+import org.apache.samza.util.Clock
+
+class TestTopicMetadataCache {
+
+  object MockTime extends Clock {
+    var currentValue = 0
+
+    def currentTimeMillis: Long = currentValue
+  }
+
+  class MockTopicMetadataStore extends TopicMetadataStore {
+    var mockCache = Map(
+      "topic1" -> new TopicMetadata("topic1", List.empty, 0),
+      "topic2" -> new TopicMetadata("topic2", List.empty, 0))
+    var numberOfCalls: AtomicInteger = new AtomicInteger(0)
+
+    def getTopicInfo(topics: Set[String]) = {
+      var topicMetadata = Map[String, TopicMetadata]()
+      topics.foreach(topic => topicMetadata += topic -> mockCache(topic))
+      numberOfCalls.getAndIncrement
+      topicMetadata
+    }
+    /*
+    def onTopicMissingMock(topic: String, zkClient: ZkClient) {
+      mockCache += topic -> new TopicMetadata(topic, List.empty, 0)
+    }
+    */
+
+    def setErrorCode(topic: String, errorCode: Short) {
+      mockCache += topic -> new TopicMetadata(topic, List.empty, errorCode)
+    }
+    def resetNumOfCalls = numberOfCalls = new AtomicInteger(0)
+  }
+
+  val mockStore = new MockTopicMetadataStore()
+  val waitForThreadStart = new CountDownLatch(3)
+
+  @Test
+  def testBasicMetadataCacheFunctionality {
+    // Retrieve a topic from the cache. Initially cache is empty and store is queried to get the data
+    mockStore.setErrorCode("topic1", 3)
+    var metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, () => MockTime.currentTimeMillis)
+    assert(metadata("topic1").topic.equals("topic1"))
+    assert(metadata("topic1").errorCode == 3)
+    assert(mockStore.numberOfCalls.get() == 1)
+
+    // Retrieve the same topic from the cache which has an error code. Ensure the store is called to refresh the cache
+    MockTime.currentValue = 5
+    mockStore.setErrorCode("topic1", 0)
+    metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, () => MockTime.currentTimeMillis)
+    assert(metadata("topic1").topic.equals("topic1"))
+    assert(metadata("topic1").errorCode == 0)
+    assert(mockStore.numberOfCalls.get() == 2)
+
+    // Retrieve the same topic from the cache with refresh rate greater than the last update. Ensure the store is not
+    // called
+    metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, () => MockTime.currentTimeMillis)
+    assert(metadata("topic1").topic.equals("topic1"))
+    assert(metadata("topic1").errorCode == 0)
+    assert(mockStore.numberOfCalls.get() == 2)
+
+    // Ensure that refresh happens when refresh rate is less than the last update. Ensure the store is called
+    MockTime.currentValue = 11
+    metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, () => MockTime.currentTimeMillis)
+    assert(metadata("topic1").topic.equals("topic1"))
+    assert(metadata("topic1").errorCode == 0)
+    assert(mockStore.numberOfCalls.get() == 3)
+  }
+
+  @Test
+  def testMultiThreadedInteractionForTopicMetadataCache {
+    mockStore.resetNumOfCalls
+    MockTime.currentValue = 17
+    val numAssertionSuccess = new AtomicBoolean(true)
+    // Add topic to the cache from multiple threads and ensure the store is called only once
+    val threads = new Array[Thread](3)
+    for (i <- 0 until 3) {
+      threads(i) = new Thread(new Runnable {
+        def run {
+          waitForThreadStart.countDown()
+          waitForThreadStart.await()
+          val metadata = TopicMetadataCache.getTopicMetadata(Set("topic1"), "kafka", mockStore.getTopicInfo, 5, () => MockTime.currentTimeMillis)
+          numAssertionSuccess.compareAndSet(true, metadata("topic1").topic.equals("topic1"))
+          numAssertionSuccess.compareAndSet(true, metadata("topic1").errorCode == 0)
+        }
+      })
+      threads(i).start()
+    }
+    for (i <- 0 until 3) {
+      threads(i).join
+    }
+    assert(numAssertionSuccess.get() == true)
+    assert(mockStore.numberOfCalls.get() == 1)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/java/org/apache/samza/storage/kv/Entry.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/Entry.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/Entry.java
new file mode 100644
index 0000000..00e9f12
--- /dev/null
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/Entry.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv;
+
+/**
+ * A key and value.
+ * 
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public class Entry<K, V> {
+
+  private final K key;
+  private final V value;
+  
+  public Entry(K key, V value) {
+    this.key = key;
+    this.value = value;
+  }
+
+  public K getKey() {
+    return key;
+  }
+
+  public V getValue() {
+    return value;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueIterator.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueIterator.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueIterator.java
new file mode 100644
index 0000000..2fb26e2
--- /dev/null
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueIterator.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv;
+
+import java.util.Iterator;
+
+public interface KeyValueIterator<K,V> extends Iterator<Entry<K,V>> {
+  public void close();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
new file mode 100644
index 0000000..49089df
--- /dev/null
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv;
+
+import java.util.List;
+
+/**
+ * A key-value store that supports put/get/delete and range queries
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface KeyValueStore<K, V> {
+  
+  /**
+   * Get the value corresponding to this key
+   * @param key The key to fetch
+   * @return The value or null if no value is found.
+   */
+  public V get(K key);
+  
+  /**
+   * Update the value associated with this key
+   * @param key They key to associate the value to
+   * @param value The value
+   */
+  public void put(K key, V value);
+  
+  /**
+   * Update all the given key/value pairs
+   */
+  public void putAll(List<Entry<K,V>> entries);
+  
+  /**
+   * Delete the value from the store (if there is one)
+   * @param key The key
+   */
+  public void delete(K key);
+  
+  /**
+   * Get an iterator over a given range of keys. This iterator MUST be closed after use.
+   * @param from The first key that could be in the range
+   * @param to The last key that could be in the range
+   * @return The iterator for this range.
+   */
+  public KeyValueIterator<K,V> range(K from, K to);
+  
+  /**
+   * Return an iterator over all keys in the database. This iterator MUST be closed after use.
+   */
+  public KeyValueIterator<K,V> all();
+  
+  public void close();
+  
+  public void flush();
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/CachedStore.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/CachedStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/CachedStore.scala
new file mode 100644
index 0000000..0eed8fa
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/CachedStore.scala
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import scala.collection._
+import grizzled.slf4j.Logging
+
+/**
+ * A write-behind caching layer around the leveldb store. The purpose of this cache is three-fold:
+ * 1. Batch together writes to leveldb, this turns out to be a great optimization
+ * 2. Avoid duplicate writes and duplicate log entries within a commit interval. i.e. if there are two updates to the same key, log only the later.
+ * 3. Avoid deserialization cost for gets on very common keys
+ *
+ * This caching does introduce a few odd corner cases :-(
+ * 1. Items in the cache have pass-by-reference semantics but items in leveldb have pass-by-value semantics. Modifying items after a put is a bad idea.
+ * 2. Range queries require flushing the cache (as the ordering comes from leveldb)
+ *
+ * In implementation this cache is just an LRU hash map that discards the oldest entry when full. There is an accompanying "dirty list" that references keys
+ * that have not yet been written to disk. All writes go to the dirty list and when the list is long enough we flush out all those values at once. Dirty items
+ * that time out of the cache before being written will also trigger a full flush of the dirty list.
+ *
+ * This class is very non-thread safe.
+ *
+ * @param store The store to cache
+ * @param cacheEntries The number of entries to hold in the in memory-cache
+ * @param writeBatchSize The number of entries to batch together before forcing a write
+ */
+class CachedStore[K, V](val store: KeyValueStore[K, V],
+  val cacheSize: Int,
+  val writeBatchSize: Int) extends KeyValueStore[K, V] with Logging {
+
+  /** the number of items in the dirty list */
+  private var dirtyCount = 0
+
+  /** the list of items to be written out on flush from newest to oldest */
+  private var dirty = new mutable.DoubleLinkedList[K]()
+
+  /** an lru cache of values that holds cacheEntries and calls flush() if necessary when discarding */
+  private val cache = new java.util.LinkedHashMap[K, CacheEntry[K, V]]((cacheSize * 1.2).toInt, 1.0f, true) {
+    override def removeEldestEntry(eldest: java.util.Map.Entry[K, CacheEntry[K, V]]): Boolean = {
+      val entry = eldest.getValue
+      // if this entry hasn't been written out yet, flush it and all other dirty keys
+      if (entry.dirty != null) {
+        debug("Found a dirty entry. Flushing.")
+
+        flush()
+      }
+      super.size > cacheSize
+    }
+  }
+
+  def get(key: K) = {
+    val c = cache.get(key)
+    if (c != null) {
+      c.value
+    } else {
+      val v = store.get(key)
+      cache.put(key, new CacheEntry(v, null))
+      v
+    }
+  }
+
+  def range(from: K, to: K) = {
+    flush()
+    store.range(from, to)
+  }
+
+  def all() = {
+    flush()
+    store.all()
+  }
+
+  def put(key: K, value: V) {
+    // add the key to the front of the dirty list (and remove any prior occurrences to dedupe)
+    val found = cache.get(key)
+    if (found == null || found.dirty == null)
+      this.dirtyCount += 1
+    else
+      found.dirty.remove()
+    this.dirty = new mutable.DoubleLinkedList(key, this.dirty)
+
+    // add the key to the cache (but don't allocate a new cache entry if we already have one)
+    if (found == null) {
+      cache.put(key, new CacheEntry(value, this.dirty))
+    } else {
+      found.value = value
+      found.dirty = this.dirty
+    }
+
+    // flush the dirty values if the write list is full
+    if (dirtyCount >= writeBatchSize) {
+      debug("Dirty count %s >= write batch size %s. Flushing." format (dirtyCount, writeBatchSize))
+
+      flush()
+    }
+  }
+
+  def flush() {
+    trace("Flushing.")
+
+    // write out the contents of the dirty list oldest first
+    val batch = new java.util.ArrayList[Entry[K, V]](this.dirtyCount)
+    for (k <- this.dirty.reverse) {
+      val entry = this.cache.get(k)
+      entry.dirty = null // not dirty any more
+      batch.add(new Entry(k, entry.value))
+    }
+    store.putAll(batch)
+    store.flush
+
+    // reset the dirty list
+    this.dirty = new mutable.DoubleLinkedList[K]()
+    this.dirtyCount = 0
+  }
+
+  /**
+   * Perform multiple local updates and log out all changes to the changelog
+   */
+  def putAll(entries: java.util.List[Entry[K, V]]) {
+    val iter = entries.iterator
+    while (iter.hasNext) {
+      val curr = iter.next
+      put(curr.getKey, curr.getValue)
+    }
+  }
+
+  /**
+   * Perform the local delete and log it out to the changelog
+   */
+  def delete(key: K) {
+    put(key, null.asInstanceOf[V])
+  }
+
+  def close() {
+    trace("Closing.")
+
+    flush
+    
+    store.close
+  }
+
+}
+
+private case class CacheEntry[K, V](var value: V, var dirty: mutable.DoubleLinkedList[K])

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala
new file mode 100644
index 0000000..92eee38
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.io.File
+import java.nio.ByteBuffer
+import org.apache.samza._
+import org.apache.samza.config.Config
+import org.apache.samza.serializers._
+import scala.collection.JavaConversions._
+import grizzled.slf4j.Logging
+import org.apache.samza.system.IncomingMessageEnvelope
+import org.apache.samza.storage.StorageEngine
+
+/**
+ * A key value store.
+ *
+ * This implements both the key/value interface and the storage engine interface.
+ */
+class KeyValueStorageEngine[K, V](
+  db: KeyValueStore[K, V],
+  rawDb: KeyValueStore[Array[Byte], Array[Byte]],
+  batchSize: Int = 500) extends StorageEngine with KeyValueStore[K, V] with Logging {
+
+  var count = 0
+
+  /* delegate to underlying store */
+  def get(key: K): V = db.get(key)
+  def put(key: K, value: V) = db.put(key, value)
+  def putAll(entries: java.util.List[Entry[K, V]]) = db.putAll(entries)
+  def delete(key: K) = db.delete(key)
+  def range(from: K, to: K) = db.range(from, to)
+  def all() = db.all()
+
+  /**
+   * Restore the contents of this key/value store from the change log,
+   * batching updates and skipping serialization for efficiency.
+   */
+  def restore(envelopes: java.util.Iterator[IncomingMessageEnvelope]) {
+    val batch = new java.util.ArrayList[Entry[Array[Byte], Array[Byte]]](batchSize)
+
+    for (envelope <- envelopes) {
+      batch.add(new Entry(envelope.getKey.asInstanceOf[Array[Byte]], envelope.getMessage.asInstanceOf[Array[Byte]]))
+
+      if (batch.size >= batchSize) {
+        rawDb.putAll(batch)
+        batch.clear()
+      }
+
+      count += 1
+
+      if (count % 1000000 == 0) {
+        info(count + " entries restored...")
+      }
+    }
+
+    if (batch.size > 0) {
+      rawDb.putAll(batch)
+    }
+  }
+
+  def flush() = {
+    trace("Flushing.")
+
+    db.flush
+  }
+
+  def stop() = {
+    trace("Stopping.")
+
+    close
+  }
+
+  def close() = {
+    trace("Closing.")
+
+    flush
+    db.close
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineFactory.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineFactory.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineFactory.scala
new file mode 100644
index 0000000..dbdefa0
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngineFactory.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.io.File
+import org.apache.samza.config.Config
+import org.apache.samza.serializers._
+import org.apache.samza.SamzaException
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.task.MessageCollector
+import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.storage.StorageEngineFactory
+import org.apache.samza.storage.StorageEngine
+
+class KeyValueStorageEngineFactory[K, V] extends StorageEngineFactory[K, V] {
+  def getStorageEngine(
+    storeName: String,
+    storeDir: File,
+    keySerde: Serde[K],
+    msgSerde: Serde[V],
+    collector: MessageCollector,
+    config: Config,
+    registry: MetricsRegistry,
+    changeLogSystemStreamPartition: SystemStreamPartition): StorageEngine = {
+
+    val storageConfig = config.subset("stores." + storeName + ".", true)
+    val batchSize = config.getInt("batch.size", 500)
+    val cacheSize = config.getInt("cache.size", math.max(batchSize, 1000))
+    val enableCache = cacheSize > 0
+
+    if (cacheSize > 0 && cacheSize < batchSize) {
+      throw new SamzaException("A stores cache.size cannot be less than batch.size as batched values reside in cache.")
+    }
+
+    if (keySerde == null) {
+      throw new SamzaException("Must define a key serde when using key value storage.")
+    }
+
+    if (msgSerde == null) {
+      throw new SamzaException("Must define a message serde when using key value storage.")
+    }
+
+    val levelDb = new LevelDbKeyValueStore(storeDir, LevelDbKeyValueStore.options(storageConfig))
+    val maybeLoggedStore = if (changeLogSystemStreamPartition == null) {
+      levelDb
+    } else {
+      new LoggedStore(levelDb, changeLogSystemStreamPartition, collector)
+    }
+    val serialized = new SerializedKeyValueStore[K, V](maybeLoggedStore, keySerde, msgSerde)
+    val maybeCachedStore = if (enableCache) {
+      new CachedStore(serialized, cacheSize, batchSize)
+    } else {
+      serialized
+    }
+    val db = maybeCachedStore
+
+    // Decide if we should use raw bytes when restoring
+
+    new KeyValueStorageEngine(db, levelDb, batchSize)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/LevelDbKeyValueStore.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/LevelDbKeyValueStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LevelDbKeyValueStore.scala
new file mode 100644
index 0000000..ac6da25
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LevelDbKeyValueStore.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.nio.ByteBuffer
+import org.iq80.leveldb._
+import org.fusesource.leveldbjni.internal.NativeComparator
+import org.fusesource.leveldbjni.JniDBFactory._
+import java.io._
+import java.util.Iterator
+import java.lang.Iterable
+import org.apache.samza.config.Config
+import grizzled.slf4j.Logging
+
+object LevelDbKeyValueStore {
+  def options(config: Config) = {
+    val options = new Options
+    options.blockSize(config.getInt("block.size", 4086))
+    options.cacheSize(config.getLong("cache.size", 64 * 1024 * 1024L))
+    options.compressionType(if (config.getBoolean("compress", true)) CompressionType.SNAPPY else CompressionType.NONE)
+    options.createIfMissing(true)
+    options.errorIfExists(true)
+    options.writeBufferSize(config.getInt("write.buffer.size", 16 * 1024 * 1024))
+    options
+  }
+}
+
+class LevelDbKeyValueStore(
+  val dir: File,
+  val options: Options) extends KeyValueStore[Array[Byte], Array[Byte]] with Logging {
+
+  private lazy val db = factory.open(dir, options)
+  private val lexicographic = new LexicographicComparator()
+
+  def get(key: Array[Byte]): Array[Byte] = {
+    require(key != null, "Null key not allowed.")
+    val found = db.get(key)
+    if (found == null)
+      null
+    else
+      found
+  }
+
+  def put(key: Array[Byte], value: Array[Byte]) {
+    require(key != null, "Null key not allowed.")
+    if (value == null)
+      db.delete(key)
+    else
+      db.put(key, value)
+  }
+
+  def putAll(entries: java.util.List[Entry[Array[Byte], Array[Byte]]]) {
+    val batch = db.createWriteBatch()
+    val iter = entries.iterator
+    while (iter.hasNext) {
+      val curr = iter.next()
+      if (curr.getValue == null)
+        batch.delete(curr.getKey)
+      else
+        batch.put(curr.getKey, curr.getValue)
+    }
+    db.write(batch)
+  }
+
+  def delete(key: Array[Byte]) {
+    put(key, null)
+  }
+
+  def range(from: Array[Byte], to: Array[Byte]): KeyValueIterator[Array[Byte], Array[Byte]] = {
+    require(from != null && to != null, "Null bound not allowed.")
+    new LevelDbRangeIterator(db.iterator, from, to)
+  }
+
+  def all(): KeyValueIterator[Array[Byte], Array[Byte]] = {
+    val iter = db.iterator()
+    iter.seekToFirst()
+    new LevelDbIterator(iter)
+  }
+
+  def flush {
+    // TODO can't find a flush for leveldb
+    trace("Flushing, but flush in LevelDbKeyValueStore doesn't do anything.")
+  }
+
+  def close() {
+    trace("Closing.")
+
+    db.close()
+  }
+
+  class LevelDbIterator(iter: DBIterator) extends KeyValueIterator[Array[Byte], Array[Byte]] {
+    private var open = true
+    def close() = {
+      open = false
+      iter.close()
+    }
+    def remove() = iter.remove()
+    def hasNext() = iter.hasNext()
+    def next() = {
+      val curr = iter.next()
+      new Entry(curr.getKey, curr.getValue)
+    }
+    override def finalize() {
+      if (open) {
+        System.err.println("Leaked reference to level db iterator, forcing close.")
+        close()
+      }
+    }
+  }
+
+  class LevelDbRangeIterator(iter: DBIterator, from: Array[Byte], to: Array[Byte]) extends LevelDbIterator(iter) {
+    val comparator = if (options.comparator == null) lexicographic else options.comparator
+    iter.seek(from)
+    override def hasNext() = {
+      iter.hasNext() && comparator.compare(iter.peekNext.getKey, to) <= 0
+    }
+  }
+
+  /**
+   * Compare two array lexicographically using unsigned byte arithmetic
+   */
+  class LexicographicComparator extends DBComparator {
+    def compare(k1: Array[Byte], k2: Array[Byte]): Int = {
+      val l = math.min(k1.length, k2.length)
+      var i = 0
+      while (i < l) {
+        if (k1(i) != k2(i))
+          return (k1(i) & 0xff) - (k2(i) & 0xff)
+        i += 1
+      }
+      // okay prefixes are equal, the shorter array is less
+      k1.length - k2.length
+    }
+    def name(): String = "lexicographic"
+    def findShortestSeparator(start: Array[Byte], limit: Array[Byte]) = start
+    def findShortSuccessor(key: Array[Byte]) = key
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala
new file mode 100644
index 0000000..a1b547d
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/LoggedStore.scala
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.nio.ByteBuffer
+import org.apache.samza.system.SystemStream
+import org.apache.samza.task.MessageCollector
+import org.apache.samza.system.OutgoingMessageEnvelope
+import org.apache.samza.system.SystemStreamPartition
+import grizzled.slf4j.Logging
+
+/**
+ * A key/value store decorator that adds a changelog for any changes made to the underlying store
+ */
+class LoggedStore[K,V](val store: KeyValueStore[K, V],
+                       val systemStreamPartition: SystemStreamPartition,
+                       val collector: MessageCollector) extends KeyValueStore[K, V] with Logging {
+
+  val systemStream = systemStreamPartition.getSystemStream
+  val partitionId = systemStreamPartition.getPartition.getPartitionId
+
+  /* pass through methods */
+  def get(key: K) = store.get(key)
+  def range(from: K, to: K) = store.range(from, to)
+  def all() = store.all()
+  
+  /**
+   * Perform the local update and log it out to the changelog
+   */
+  def put(key: K, value: V) {
+    store.put(key, value)
+    collector.send(new OutgoingMessageEnvelope(systemStream, partitionId, key, value))
+  }
+  
+  /**
+   * Perform multiple local updates and log out all changes to the changelog
+   */
+  def putAll(entries: java.util.List[Entry[K,V]]) {
+    store.putAll(entries)
+    val iter = entries.iterator
+    while(iter.hasNext) {
+      val curr = iter.next
+      collector.send(new OutgoingMessageEnvelope(systemStream, partitionId, curr.getKey, curr.getValue))
+    }
+  }
+  
+  /**
+   * Perform the local delete and log it out to the changelog 
+   */
+  def delete(key: K) {
+    store.delete(key)
+    collector.send(new OutgoingMessageEnvelope(systemStream, partitionId, key, null))
+  }
+  
+  def flush {
+    trace("Flushing.")
+
+    store.flush
+  }
+  
+  def close {
+    trace("Closing.")
+
+    store.close
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala
new file mode 100644
index 0000000..75fd414
--- /dev/null
+++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.util.Iterator
+import org.apache.samza.serializers._
+import grizzled.slf4j.Logging
+
+/**
+ * A key-value store wrapper that handles serialization
+ */
+class SerializedKeyValueStore[K, V](
+  store: KeyValueStore[Array[Byte], Array[Byte]],
+  keySerde: Serde[K],
+  msgSerde: Serde[V]) extends KeyValueStore[K, V] with Logging {
+
+  def get(key: K): V = {
+    val keyBytes = keySerde.toBytes(key)
+    val found = store.get(keyBytes)
+    if (found == null)
+      null.asInstanceOf[V]
+    else
+      msgSerde.fromBytes(found).asInstanceOf[V]
+  }
+
+  def put(key: K, value: V) {
+    val keyBytes = keySerde.toBytes(key)
+    val valBytes = msgSerde.toBytes(value)
+    store.put(keyBytes, valBytes)
+  }
+
+  def putAll(entries: java.util.List[Entry[K, V]]) {
+    val list = new java.util.ArrayList[Entry[Array[Byte], Array[Byte]]](entries.size())
+    val iter = entries.iterator
+    while (iter.hasNext) {
+      val curr = iter.next
+      val keyBytes = keySerde.toBytes(curr.getKey)
+      val valBytes = msgSerde.toBytes(curr.getValue)
+      list.add(new Entry(keyBytes, valBytes))
+    }
+    store.putAll(list)
+  }
+
+  def delete(key: K) {
+    store.delete(keySerde.toBytes(key))
+  }
+
+  def range(from: K, to: K): KeyValueIterator[K, V] = {
+    val fromBytes = keySerde.toBytes(from)
+    val toBytes = keySerde.toBytes(to)
+    new DeserializingIterator(store.range(fromBytes, toBytes))
+  }
+
+  def all(): KeyValueIterator[K, V] =
+    new DeserializingIterator(store.all)
+
+  private class DeserializingIterator(iter: KeyValueIterator[Array[Byte], Array[Byte]]) extends KeyValueIterator[K, V] {
+    def hasNext() = iter.hasNext()
+    def remove() = iter.remove()
+    def close() = iter.close()
+    def next(): Entry[K, V] = {
+      val nxt = iter.next()
+      new Entry(keySerde.fromBytes(nxt.getKey()).asInstanceOf[K], msgSerde.fromBytes(nxt.getValue()).asInstanceOf[V])
+    }
+  }
+
+  def flush {
+    trace("Flushing.")
+
+    store.flush
+  }
+
+  def close {
+    trace("Closing.")
+
+    store.close
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStores.scala
----------------------------------------------------------------------
diff --git a/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStores.scala b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStores.scala
new file mode 100644
index 0000000..5f9d66c
--- /dev/null
+++ b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStores.scala
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage.kv
+
+import java.io.File
+import java.util.Arrays
+import java.util.Random
+
+import scala.collection.JavaConversions._
+
+import org.iq80.leveldb.Options
+import org.junit.After
+import org.junit.Assert.assertEquals
+import org.junit.Assert.assertFalse
+import org.junit.Assert.assertNull
+import org.junit.Assert.assertTrue
+import org.junit.Before
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.runners.Parameterized.Parameters
+
+@RunWith(value = classOf[Parameterized])
+class TestKeyValueStores(cache: Boolean) {
+
+  import TestKeyValueStores._
+
+  val letters = "abcdefghijklmnopqrstuvwxyz".map(_.toString)
+  val dir = new File(System.getProperty("java.io.tmpdir"), "leveldb-test-" + new Random().nextInt(Int.MaxValue))
+  var store: KeyValueStore[Array[Byte], Array[Byte]] = null
+
+  @Before
+  def setup() {
+    dir.mkdirs()
+    val leveldb = new LevelDbKeyValueStore(dir, new Options)
+    if (cache)
+      store = new CachedStore(leveldb, CacheSize, BatchSize)
+    else
+      store = leveldb
+  }
+
+  @After
+  def teardown() {
+    for (file <- dir.listFiles)
+      file.delete()
+    dir.delete()
+  }
+
+  @Test
+  def getNonExistantIsNull() {
+    assertNull(store.get(b("hello")))
+  }
+
+  @Test
+  def putAndGet() {
+    store.put(b("k"), b("v"))
+    assertTrue(Arrays.equals(b("v"), store.get(b("k"))))
+  }
+
+  @Test
+  def doublePutAndGet() {
+    val k = b("k2")
+    store.put(k, b("v1"))
+    store.put(k, b("v2"))
+    assertTrue(Arrays.equals(b("v2"), store.get(k)))
+  }
+
+  @Test
+  def testPutAll() {
+    // Use CacheSize - 1 so we fully fill the cache, but don't write any data 
+    // out. Our check (below) uses == for cached entries, and using 
+    // numEntires >= CacheSize would result in the LRU cache dropping some 
+    // entries. The result would be that we get the correct byte array back 
+    // from the cache's underlying store (leveldb), but that == would fail.
+    val numEntries = CacheSize - 1
+    val entries = (0 until numEntries).map(i => new Entry(b("k" + i), b("v" + i)))
+    store.putAll(entries)
+    if (cache) {
+      assertTrue("All values should be found and cached.", entries.forall(e => store.get(e.getKey) == e.getValue))
+    } else {
+      assertTrue("All values should be found.", entries.forall(e => Arrays.equals(store.get(e.getKey), e.getValue)))
+    }
+  }
+
+  @Test
+  def testIterateAll() {
+    for (letter <- letters)
+      store.put(b(letter.toString), b(letter.toString))
+    val iter = store.all
+    checkRange(letters, iter)
+    iter.close()
+  }
+
+  @Test
+  def testRange() {
+    val from = 5
+    val to = 20
+    for (letter <- letters)
+      store.put(b(letter.toString), b(letter.toString))
+    val iter = store.range(b(letters(from)), b(letters(to)))
+    checkRange(letters.slice(from, to + 1), iter)
+    iter.close()
+  }
+
+  @Test
+  def testDelete() {
+    val a = b("a")
+    store.put(a, a)
+    store.delete(a)
+    assertNull(store.get(a))
+  }
+
+  @Test
+  def testSimpleScenario() {
+    val vals = letters.map(b(_))
+    for (v <- vals) {
+      assertNull(store.get(v))
+      store.put(v, v)
+      assertTrue(Arrays.equals(v, store.get(v)))
+    }
+    vals.foreach(v => assertTrue(Arrays.equals(v, store.get(v))))
+    vals.foreach(v => store.delete(v))
+    vals.foreach(v => assertNull(store.get(v)))
+  }
+
+  def checkRange(vals: IndexedSeq[String], iter: KeyValueIterator[Array[Byte], Array[Byte]]) {
+    for (v <- vals) {
+      assertTrue(iter.hasNext)
+      val entry = iter.next()
+      assertEquals(v, s(entry.getKey))
+      assertEquals(v, s(entry.getValue))
+    }
+    assertFalse(iter.hasNext)
+  }
+
+  /**
+   * Convert string to byte buffer
+   */
+  def b(s: String) =
+    s.getBytes
+
+  /**
+   * Convert byte buffer to string
+   */
+  def s(b: Array[Byte]) =
+    new String(b)
+
+}
+
+object TestKeyValueStores {
+  val CacheSize = 10
+  val BatchSize = 2
+  @Parameters
+  def parameters: java.util.Collection[Array[java.lang.Boolean]] = Arrays.asList(Array(java.lang.Boolean.TRUE), Array(java.lang.Boolean.FALSE))
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-serializers/src/main/scala/org/apache/samza/serializers/JsonSerde.scala
----------------------------------------------------------------------
diff --git a/samza-serializers/src/main/scala/org/apache/samza/serializers/JsonSerde.scala b/samza-serializers/src/main/scala/org/apache/samza/serializers/JsonSerde.scala
new file mode 100644
index 0000000..744eec0
--- /dev/null
+++ b/samza-serializers/src/main/scala/org/apache/samza/serializers/JsonSerde.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import org.codehaus.jackson.map.ObjectMapper
+import java.nio.ByteBuffer
+import org.apache.samza.config.Config
+
+class JsonSerde extends Serde[Object] {
+  val objectMapper = new ObjectMapper
+
+  def toBytes(obj: Object) = objectMapper
+    .writeValueAsString(obj)
+    .getBytes("UTF-8")
+
+  def fromBytes(bytes: Array[Byte]) = objectMapper
+    .readValue(bytes, classOf[Object])
+}
+
+class JsonSerdeFactory extends SerdeFactory[Object] {
+  def getSerde(name: String, config: Config) = new JsonSerde
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-serializers/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerde.scala
----------------------------------------------------------------------
diff --git a/samza-serializers/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerde.scala b/samza-serializers/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerde.scala
new file mode 100644
index 0000000..455dd34
--- /dev/null
+++ b/samza-serializers/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerde.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import org.apache.samza.config.Config
+import org.codehaus.jackson.map.ObjectMapper
+import java.util.Map
+import java.nio.ByteBuffer
+import org.apache.samza.metrics.reporter.MetricsSnapshot
+
+class MetricsSnapshotSerde extends Serde[MetricsSnapshot] {
+  val jsonMapper = new ObjectMapper
+
+  def toBytes(obj: MetricsSnapshot) = jsonMapper
+    .writeValueAsString(obj.getAsMap)
+    .getBytes("UTF-8")
+
+  def fromBytes(bytes: Array[Byte]) = {
+    val metricMap = jsonMapper.readValue(bytes, classOf[java.util.Map[String, java.util.Map[String, Object]]])
+    MetricsSnapshot.fromMap(metricMap)
+  }
+}
+
+class MetricsSnapshotSerdeFactory extends SerdeFactory[MetricsSnapshot] {
+  def getSerde(name: String, config: Config) = new MetricsSnapshotSerde
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-serializers/src/test/scala/org/apache/samza/serializers/TestJsonSerde.scala
----------------------------------------------------------------------
diff --git a/samza-serializers/src/test/scala/org/apache/samza/serializers/TestJsonSerde.scala b/samza-serializers/src/test/scala/org/apache/samza/serializers/TestJsonSerde.scala
new file mode 100644
index 0000000..e2a153b
--- /dev/null
+++ b/samza-serializers/src/test/scala/org/apache/samza/serializers/TestJsonSerde.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import org.junit.Assert._
+import org.junit.Test
+import scala.collection.JavaConversions._
+
+class TestJsonSerde {
+  @Test
+  def testJsonSerdeShouldWork {
+    val serde = new JsonSerde
+    val obj = new java.util.HashMap[String, Object](Map[String, Object]("hi" -> "bye", "why" -> new java.lang.Integer(2)))
+    val bytes = serde.toBytes(obj)
+    serde.fromBytes(bytes).equals(obj)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-serializers/src/test/scala/org/apache/samza/serializers/TestMetricsSnapshotSerde.scala
----------------------------------------------------------------------
diff --git a/samza-serializers/src/test/scala/org/apache/samza/serializers/TestMetricsSnapshotSerde.scala b/samza-serializers/src/test/scala/org/apache/samza/serializers/TestMetricsSnapshotSerde.scala
new file mode 100644
index 0000000..b307334
--- /dev/null
+++ b/samza-serializers/src/test/scala/org/apache/samza/serializers/TestMetricsSnapshotSerde.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.metrics.reporter.MetricsSnapshot
+import org.apache.samza.metrics.reporter.MetricsHeader
+import org.apache.samza.metrics.reporter.Metrics
+import java.util.HashMap
+import java.util.Map
+
+class TestMetricsSnapshotSerde {
+  @Test
+  def testMetricsSerdeShouldSerializeAndDeserializeAMetric {
+    val header = new MetricsHeader("test", "testjobid", "task", "test", "version", "samzaversion", "host", 1L, 2L)
+    val metricsMap = new HashMap[String, Object]()
+    metricsMap.put("test2", "foo")
+    val metricsGroupMap = new HashMap[String, Map[String, Object]]()
+    metricsGroupMap.put("test", metricsMap)
+    val metrics = Metrics.fromMap(metricsGroupMap)
+    val snapshot = new MetricsSnapshot(header, metrics)
+    val serde = new MetricsSnapshotSerde()
+    val bytes = serde.toBytes(snapshot)
+    serde.fromBytes(bytes).equals(metrics)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/assembly/src.xml
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/assembly/src.xml b/samza-shell/src/main/assembly/src.xml
new file mode 100644
index 0000000..5173fdf
--- /dev/null
+++ b/samza-shell/src/main/assembly/src.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor 
+  license agreements. See the NOTICE file distributed with this work for additional 
+  information regarding copyright ownership. The ASF licenses this file to 
+  you 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. -->
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+  <id>dist</id>
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <outputDirectory></outputDirectory>
+      <directory>${basedir}/src/main/bash</directory>
+      <fileMode>0744</fileMode>
+      <includes>
+        <include>*</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/bash/kill-yarn-job.sh
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/bash/kill-yarn-job.sh b/samza-shell/src/main/bash/kill-yarn-job.sh
new file mode 100755
index 0000000..6105c81
--- /dev/null
+++ b/samza-shell/src/main/bash/kill-yarn-job.sh
@@ -0,0 +1,19 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+exec $(dirname $0)/run-class.sh org.apache.hadoop.yarn.client.cli.ApplicationCLI -kill $@

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/bash/run-am.sh
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/bash/run-am.sh b/samza-shell/src/main/bash/run-am.sh
new file mode 100755
index 0000000..c202596
--- /dev/null
+++ b/samza-shell/src/main/bash/run-am.sh
@@ -0,0 +1,19 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+exec $(dirname $0)/run-class.sh org.apache.samza.job.yarn.SamzaAppMaster $@

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/bash/run-class.sh
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/bash/run-class.sh b/samza-shell/src/main/bash/run-class.sh
new file mode 100755
index 0000000..e91bf99
--- /dev/null
+++ b/samza-shell/src/main/bash/run-class.sh
@@ -0,0 +1,61 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+if [ $# -lt 1 ];
+then
+  echo "USAGE: $0 classname [opts]"
+  exit 1
+fi
+
+home_dir=`pwd`
+base_dir=$(dirname $0)/..
+cd $base_dir
+base_dir=`pwd`
+cd $home_dir
+
+if [ ! -d "$base_dir/lib" ]; then
+  echo "Unable to find $base_dir/lib, which is required to run."
+  exit 1
+fi
+
+CLASSPATH=$YARN_HOME/conf
+
+for file in $base_dir/lib/*.[jw]ar;
+do
+  CLASSPATH=$CLASSPATH:$file
+done
+
+if [ -z "$JAVA_HOME" ]; then
+  JAVA="java"
+else
+  JAVA="$JAVA_HOME/bin/java"
+fi
+
+if [ -z "$SAMZA_LOG_DIR" ]; then
+  SAMZA_LOG_DIR="/tmp"
+fi
+
+if [ -z "$SAMZA_OPTS" ]; then
+  SAMZA_OPTS="-Xmx160M -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log"
+  if [ -f $base_dir/lib/log4j.xml ]; then
+    SAMZA_OPTS="$SAMZA_OPTS -Dlog4j.configuration=file:$base_dir/lib/log4j.xml"
+  fi
+fi
+
+echo $JAVA $SAMZA_OPTS -cp $CLASSPATH $@
+exec $JAVA $SAMZA_OPTS -cp $CLASSPATH $@

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/bash/run-job.sh
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/bash/run-job.sh b/samza-shell/src/main/bash/run-job.sh
new file mode 100755
index 0000000..0605994
--- /dev/null
+++ b/samza-shell/src/main/bash/run-job.sh
@@ -0,0 +1,19 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+exec $(dirname $0)/run-class.sh org.apache.samza.job.JobRunner $@


[10/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/packaging.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/packaging.md b/docs/learn/documentation/0.7.0/jobs/packaging.md
new file mode 100644
index 0000000..796d0fe
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/packaging.md
@@ -0,0 +1,31 @@
+---
+layout: page
+title: Packaging
+---
+
+The [JobRunner](job-runner.html) page talks about run-job.sh, and how it's used to start a job either locally (LocalJobFactory) or with YARN (YarnJobFactory). In the diagram that shows the execution flow, it also shows a run-task.sh script. This script, along with a run-am.sh script, are what Samza actually calls to execute its code.
+
+```
+bin/run-am.sh
+bin/run-task.sh
+```
+
+The run-task.sh script is responsible for starting the TaskRunner. The run-am.sh script is responsible for starting Samza's application master for YARN. Thus, the run-am.sh script is only used by the YarnJob, but both YarnJob and ProcessJob use run-task.sh.
+
+Typically, these two scripts are bundled into a tar.gz file that has a structure like this:
+
+```
+bin/run-am.sh
+bin/run-class.sh
+bin/run-job.sh
+bin/run-task.sh
+lib/*.jar
+```
+
+To run a Samza job, you un-zip its tar.gz file, and execute the run-job.sh script, as defined in the JobRunner section. There are a number of interesting implications from this packaging scheme. First, you'll notice that there is no configuration in the package. Second, you'll notice that the lib directory contains all JARs that you'll need to run your Samza job.
+
+The reason that configuration is decoupled from your Samza job packaging is that it allows configuration to be updated without having to re-build the entire Samza package. This makes life easier for everyone when you just need to tweak one parameter, and don't want to have to worry about which branch your package was built from, or whether trunk is in a stable state. It also has the added benefit of forcing configuration to be fully resolved at runtime. This means that that the configuration for a job is resolved at the time run-job.sh is called (using --config-path and --config-provider parameters), and from that point on, the configuration is immutable, and passed where it needs to be by Samza (and YARN, if you're using it).
+
+The second statement, that your Samza package contains all JARs that it needs to run, means that a Samza package is entirely self contained. This allows Samza jobs to run on independent Samza versions without conflicting with each other. This is in contrast to Hadoop, where JARs are pulled in from the local machine that the job is running on (using environment variables). With Samza, you might run your job on version 0.7.0, and someone else might run their job on version 0.8.0. There is no problem with this.
+
+## [YARN Jobs &raquo;](yarn-jobs.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md b/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
new file mode 100644
index 0000000..93e8610
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
@@ -0,0 +1,16 @@
+---
+layout: page
+title: YARN Jobs
+---
+
+When you define job.factory.class=samza.job.yarn.YarnJobFactory in your job's configuration, Samza will use YARN to execute your job. The YarnJobFactory will use the YARN_HOME environment variable on the machine that run-job.sh is executed on to get the appropriate YARN configuration, which will define where the YARN resource manager is. The YarnJob will work with the resource manager to get your job started on the YARN cluster.
+
+If you want to use YARN to run your Samza job, you'll also need to define the location of your Samza job's package. For example, you might say:
+
+```
+yarn.package.path=http://my.http.server/jobs/ingraphs-package-0.0.55.tgz
+```
+
+This .tgz file follows the conventions outlined on the [Packaging](packaging.html) page (it has bin/run-am.sh and bin/run-task.sh). YARN NodeManagers will take responsibility for downloading this .tgz file on the appropriate machines, and untar'ing them. From there, YARN will execute run-am.sh or run-task.sh for the Samza Application Master, and TaskRunner, respectively.
+
+## [Logging &raquo;](logging.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/operations/kafka.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/operations/kafka.md b/docs/learn/documentation/0.7.0/operations/kafka.md
new file mode 100644
index 0000000..8bb1a44
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/operations/kafka.md
@@ -0,0 +1,16 @@
+---
+layout: page
+title: Kafka
+---
+
+<!-- TODO kafka page should be fleshed out a bit -->
+
+<!-- TODO when 0.8.1 is released, update with state management config information -->
+
+Kafka has a great [operations wiki](http://kafka.apache.org/08/ops.html), which provides some detail on how to operate Kafka at scale.
+
+### Auto-Create Topics
+
+Kafka brokers should be configured to automatically create topics. Without this, it's going to be very cumbersome to run Samze jobs, since jobs will write to arbitrary (and sometimes new) topics.
+
+    auto.create.topics.enable=true

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/operations/security.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/operations/security.md b/docs/learn/documentation/0.7.0/operations/security.md
new file mode 100644
index 0000000..55a7f11
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/operations/security.md
@@ -0,0 +1,56 @@
+---
+layout: page
+title: Security
+---
+
+Samza provides no security. All security is implemented in the stream system, or in the environment that Samza containers run.
+
+### Securing Streaming Systems
+
+Samza does not provide any security at the stream system level. It is up to individual streaming systems to enforce their own security. If a stream system requires usernames and passwords in order to consume from specific streams, these values must be supplied via configuration, and used at the StreamConsumer/StreamConsumerFactory implementation. The same holds true if the streaming system uses SSL certificates or Kerberos. The environment in which Samza runs must provide the appropriate certificate or Kerberos ticket, and the StreamConsumer must be implemented to use these certificates or tickets.
+
+#### Securing Kafka
+
+Kafka provides no security for its topics, and therefore Samza doesn't provide any security when using Kafka topics.
+
+### Securing Samza's Environment
+
+The most important thing to keep in mind when securing an environment that Samza containers run in is that **Samza containers execute arbitrary user code**. They must considered an adversarial application, and the environment must be locked down accordingly.
+
+#### Configuration
+
+Samza reads all configuration at the time a Samza job is started using the run-job.sh script. If configuration contains sensitive information, then care must be taken to provide the JobRunner with the configuration. This means implementing a ConfigFactory that understands the configuration security model, and resolves configuration to Samza's Config object in a secure way.
+
+During the duration of a Samza job's execution, the configuration is kept in memory. The only time configuration is visible is:
+
+1. When configuration is resolved using a ConfigFactory.
+2. The configuration is printed to STDOUT when run-job.sh is run.
+3. The configuration is written to the logs when a Samza container starts.
+
+If configuration contains sensitive data, then these three points must be secured.
+
+#### Ports
+
+The only port that a Samza container opens by default is an un-secured JMX port that is randomly selected at start time. If this is not desired, JMX can be disabled through configuration. See the [Configuration](configuration.html) page for details.
+
+Users might open ports from inside a Samza container. If this is not desired, then the user that executes the Samza container must have the appropriate permissions revoked, usually using iptables.
+
+#### Logs
+
+Samza container logs contain configuration, and might contain arbitrary sensitive data logged by the user. A secure log directory must be provided to the Samza container.
+
+#### Starting a Samza Job
+
+If operators do not wish to allow Samza containers to be executed by arbitrary users, then the mechanism that Samza containers are deployed must secured. Usually, this means controlling execution of the run-job.sh script. The recommended pattern is to lock down the machines that Samza containers run on, and execute run-job.sh from either a blessed web service or special machine, and only allow access to the service or machine by specific users.
+
+#### Shell Scripts
+
+Please see the [Packaging](packaging.html) section for details on the the shell scripts that Samza uses. Samza containers allow users to execute arbitrary shell commands, so user permissions must be locked down to prevent users from damaging the environment or reading sensitive data.
+
+#### YARN
+
+<!-- TODO make the security page link to the actual YARN security document, when we write it. -->
+
+Samza provides out-of-the-box YARN integration. Take a look at Samza's YARN Security page for details.
+
+## [Kafka &raquo;](kafka.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/yarn/application-master.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/yarn/application-master.md b/docs/learn/documentation/0.7.0/yarn/application-master.md
new file mode 100644
index 0000000..0da6dc0
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/yarn/application-master.md
@@ -0,0 +1,53 @@
+---
+layout: page
+title: Application Master
+---
+
+YARN is Hadoop's next-generation cluster manager. It allows developers to deploy and execute arbitrary commands on a grid. If you're unfamiliar with YARN, or the concept of an ApplicationMaster (AM), please read Hadoop's [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) page.
+
+### Integration
+
+Samza's main integration with YARN comes in the form of a Samza ApplicationMaster. This is the chunk of code responsible for managing a Samza job in a YARN grid. It decides what to do when a stream processor fails, which machines a Samza job's [TaskRunner](../container/task-runner.html) should run on, and so on.
+
+When the Samza ApplicationMaster starts up, it does the following:
+
+1. Receives configuration from YARN via the STREAMING_CONFIG environment variable.
+2. Starts a JMX server on a random port.
+3. Instantiates a metrics registry and reporters to keep track of relevant metrics.
+4. Registers the AM with YARN's RM.
+5. Get the total number of partitions for the Samza job using each input stream's PartitionManager (see the [Streams](../container/streams.html) page for details).
+6. Read the total number of containers requested from the Samza job's configuration.
+7. Assign each partition to a container (called a Task Group in Samza's AM dashboard).
+8. Make a [ResourceRequest](http://hadoop.apache.org/docs/current/api/org/apache/hadoop/yarn/api/records/ResourceRequest.html) to YARN for each container.
+9. Poll the YARN RM every second to check for allocated and released containers.
+
+From this point on, the ApplicationMaster just reacts to events from the RM.
+
+### Fault Tolerance
+
+Whenever a container is allocated, the AM will work with the YARN NM to start a TaskRunner (with appropriate partitions assigned to it) in the container. If a container fails with a non-zero return code, the AM will request a new container, and restart the TaskRunner. If a TaskRunner fails too many times, too quickly, the ApplicationMaster will fail the whole Samza job with a non-zero return code. See the yarn.countainer.retry.count and yarn.container.retry.window.ms [configuration](../jobs/configuration.html) parameters for details.
+
+When the AM receives a reboot signal from YARN, it will throw a SamzaException. This will trigger a clean and successful shutdown of the AM (YARN won't think the AM failed).
+
+If the AM, itself, fails, YARN will handle restarting the AM. When the AM is restarted, all containers that were running will be killed, and the AM will start from scratch. The same list of operations, shown above, will be executed. The AM will request new containers for its TaskRunners, and proceed as though it has just started for the first time. YARN has a yarn.resourcemanager.am.max-retries configuration parameter that's defined in [yarn-site.xml](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-common/yarn-default.xml). This configuration defaults to 1, which means that, by default, a single AM failure will cause your Samza job to stop running.
+
+### Dashboard
+
+Samza's ApplicationMaster comes with a dashboard to show useful information such as:
+
+1. Where containers are located.
+2. Links to logs.
+3. The Samza job's configuration.
+4. Container failure count.
+
+You can find this dashboard by going to your YARN grid's ResourceManager page (usually something like [http://localhost:8088/cluster](http://localhost:8088/cluster)), and clicking on the "ApplicationMaster" link of a running Samza job.
+
+![diagram](/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png)
+
+### Security
+
+The Samza dashboard's HTTP access is currently un-secured, even when using YARN in secure-mode. This means that users with access to a YARN grid could port-scan a Samza ApplicationMaster's HTTP server, and open the dashboard in a browser to view its contents. Sensitive configuration can be viewed by anyone, in this way, and care should be taken. There are plans to secure Samza's ApplicationMaster using [Hadoop's security](http://docs.hortonworks.com/HDPDocuments/HDP1/HDP-1.3.0/bk_installing_manually_book/content/rpm-chap14-2-3-1.html) features ([SPENAGO](http://en.wikipedia.org/wiki/SPNEGO)).
+
+See Samza's [security](../operations/security.html) page for more details.
+
+## [Isolation &raquo;](isolation.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/documentation/0.7.0/yarn/isolation.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/yarn/isolation.md b/docs/learn/documentation/0.7.0/yarn/isolation.md
new file mode 100644
index 0000000..c685729
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/yarn/isolation.md
@@ -0,0 +1,30 @@
+---
+layout: page
+title: Isolation
+---
+
+When running Samza jobs in a shared, distributed environment, the stream processors can have an impact on one another's performance. A stream processor that uses 100% of a machine's CPU will slow down all other stream processors on the machine.
+
+One of [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html)'s responsibilities is to manage resources so that this doesn't happen. Each of YARN's Node Managers (NM) has a chunk of "resources" dedicated to it. The YARN Resource Manager (RM) will only allow a container to be allocated on a NM if it has enough resources to satisfy the container's needs.
+
+YARN currently supports resource management for memory and CPU.
+
+### Memory
+
+YARN will automatically enforce memory limits for all containers that it executes. All containers must have a max-memory size defined when they're created. If the sum of all memory usage for processes associated with a single YARN container exceeds this maximum, YARN will kill the container.
+
+Samza supports memory limits using the yarn.container.memory.mb and yarn.am.container.memory.mb configuration parameters. Keep in mind that this is simply the amount of memory YARN will allow a Samza [TaskRunner](../container/task-runner.html) or [ApplicationMaster](application-master.html) to have. You'll still need to configure your heap settings appropriately using task.opts, when using Java (the default is -Xmx160M). See the [Configuration](../jobs/configuration.html) and [Packaging](../jobs/packaging.html) pages for details.
+
+### CPU
+
+YARN has the concept of a virtual core. Each NM is assigned a total number of virtual cores (32, by default). When a container request is made, it must specify how many virtual cores it needs. The YARN RM will only assign the container to a NM that has enough virtual cores to satisfy the request.
+
+#### CGroups
+
+Unlike memory, which YARN can enforce itself (by looking at the /proc folder), YARN can't enforce CPU isolation, since this must be done at the Linux kernel level. One of YARN's interesting new features is its support for Linux [CGroups](https://www.kernel.org/doc/Documentation/cgroups/cgroups.txt). CGroups are a way to control process utilization at the kernel level in Linux.
+
+If YARN is setup to use CGroups, then YARN will guarantee that a container will get at least the amount of CPU that it requires. Currently, YARN will give you more CPU, if it's available. For details on enforcing "at most" CPU usage, see [YARN-810](https://issues.apache.org/jira/browse/YARN-810). 
+
+See [this blog post](http://riccomini.name/posts/hadoop/2013-06-14-yarn-with-cgroups/) for details on setting up YARN with CGroups.
+
+## [Security &raquo;](../operations/security.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/committing.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/committing.md b/docs/learn/tutorials/0.7.0/committing.md
new file mode 100644
index 0000000..caf5b99
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/committing.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Committing
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/configuring-kafka-system.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/configuring-kafka-system.md b/docs/learn/tutorials/0.7.0/configuring-kafka-system.md
new file mode 100644
index 0000000..2e4570c
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/configuring-kafka-system.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Configuring a Kafka System
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/group-by-count.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/group-by-count.md b/docs/learn/tutorials/0.7.0/group-by-count.md
new file mode 100644
index 0000000..fe022ea
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/group-by-count.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Group-by and Counting
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/index.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/index.md b/docs/learn/tutorials/0.7.0/index.md
new file mode 100644
index 0000000..b9254ec
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/index.md
@@ -0,0 +1,17 @@
+---
+layout: page
+title: Tutorials
+---
+
+No tutorials are available right now.
+
+<!-- TODO a bunch of tutorials
+[Log Walkthrough](log-walkthrough.html)
+<a href="configuring-kafka-system.html">Configuring a Kafka System</a><br/>
+<a href="joining-streams.html">Joining Streams</a><br/>
+<a href="sort-stream.html">Sorting a Stream</a><br/>
+<a href="group-by-count.html">Group-by and Counting</a><br/>
+<a href="initialize-close.html">Initializing and Closing</a><br/>
+<a href="windowing.html">Windowing</a><br/>
+<a href="committing.html">Committing</a><br/>
+-->

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/initialize-close.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/initialize-close.md b/docs/learn/tutorials/0.7.0/initialize-close.md
new file mode 100644
index 0000000..3d2dbad
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/initialize-close.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Initializing and Closing
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/joining-streams.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/joining-streams.md b/docs/learn/tutorials/0.7.0/joining-streams.md
new file mode 100644
index 0000000..8224ebb
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/joining-streams.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Joining Streams
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/sort-stream.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/sort-stream.md b/docs/learn/tutorials/0.7.0/sort-stream.md
new file mode 100644
index 0000000..f2e6200
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/sort-stream.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Sorting a Stream
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/learn/tutorials/0.7.0/windowing.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/0.7.0/windowing.md b/docs/learn/tutorials/0.7.0/windowing.md
new file mode 100644
index 0000000..611563c
--- /dev/null
+++ b/docs/learn/tutorials/0.7.0/windowing.md
@@ -0,0 +1,6 @@
+---
+layout: page
+title: Windowing
+---
+
+haiii

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/startup/download/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md
new file mode 100644
index 0000000..f3ecfa6
--- /dev/null
+++ b/docs/startup/download/index.md
@@ -0,0 +1,81 @@
+---
+layout: page
+title: Download
+---
+
+<!-- TODO remove samza code warning when we get our git repo setup -->
+
+*NOTE: The Samza code is not available until we get an Apache git repository. Follow along [here](https://issues.apache.org/jira/browse/INFRA-6617).*
+
+<!-- TODO update maven dependency versions appropriately -->
+
+If you want to play around with Samza for the first time, go to [Hello Samza](/startup/hello-samza/0.7.0).
+
+<!--
+### Maven
+
+All Samza JARs are published through Maven.
+
+#### Artifacts
+
+A Samza project that runs with Kafka and YARN should depend on the following artifacts.
+
+    <dependency>
+      <groupId>samza</groupId>
+      <artifactId>samza-api</artifactId>
+      <version>0.7.0</version>
+    </dependency>
+    <dependency>
+      <groupId>samza</groupId>
+      <artifactId>samza-core_2.9.2</artifactId>
+      <version>0.7.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>samza</groupId>
+      <artifactId>samza-serializers_2.9.2</artifactId>
+      <version>0.7.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>samza</groupId>
+      <artifactId>samza-yarn_2.9.2</artifactId>
+      <version>0.7.0</version>
+      <classifier>yarn-2.0.5-alpha</classifier>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>samza</groupId>
+      <artifactId>samza-kafka_2.9.2</artifactId>
+      <version>0.7.0</version>
+      <scope>runtime</scope>
+    </dependency>
+
+#### Repositories
+
+Samza is available in the Apache Maven repository.
+
+    <repository>
+      <id>apache-releases</id>
+      <url>https://repository.apache.org/content/groups/public</url>
+    </repository>
+
+Snapshot builds are available in the Apache Maven snapshot repository.
+
+    <repository>
+      <id>apache-snapshots</id>
+      <url>https://repository.apache.org/content/groups/snapshots</url>
+    </repository>
+-->
+
+### Checking out and Building
+
+If you're interested in working on Samza, or building the JARs from scratch, then you'll need to checkout and build the code. Samza does not have a binary release at this time. To check out and build Samza, run these commands.
+
+```
+git clone http://git-wip-us.apache.org/repos/asf/incubator-samza.git
+cd incubator-samza
+./gradlew clean build
+```
+
+See the README.md file for details on building.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/startup/hello-samza/0.7.0/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/hello-samza/0.7.0/index.md b/docs/startup/hello-samza/0.7.0/index.md
new file mode 100644
index 0000000..716ccd4
--- /dev/null
+++ b/docs/startup/hello-samza/0.7.0/index.md
@@ -0,0 +1,84 @@
+---
+layout: page
+title: Hello Samza
+---
+
+<!-- TODO remove samza code warning when we get our git repo setup -->
+
+*NOTE: The Samza code is not available until we get an Apache git repository. Follow along [here](https://issues.apache.org/jira/browse/INFRA-6617).*
+
+<!-- TODO Make github project public when code is up. -->
+
+The [hello-samza](http://github.com/linkedin/hello-samza) project is a stand-alone project designed to help you run your first Samza job.
+
+### Get the Code
+
+Start by checking out the hello-samza project.
+
+    git clone git://github.com/linkedin/hello-samza.git
+
+This project contains everything you'll need to run your first Samza jobs.
+
+### Start a Grid
+
+A Samza grid usually comprises three different systems: [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html), [Kafka](http://kafka.apache.org/), and [ZooKeeper](http://zookeeper.apache.org/). The hello-samza project comes with a script called "grid" to help you setup these systems. Start by running:
+
+    bin/grid
+
+This command will download, install, and start ZooKeeper, Kafka, and YARN. All package files will be put in a sub-directory called "deploy" inside hello-samza's root folder.
+
+Once the grid command completes, you can verify that YARN is up and running by going to [http://localhost:8088](http://localhost:8088). This is the YARN UI.
+
+### Build a Samza Job Package
+
+Before you can run a Samza job, you need to build a package for it. This package is what YARN uses to deploy your jobs on the grid.
+
+    mvn clean package
+    mkdir -p deploy/samza
+    tar -xvf ./samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz -C deploy/samza
+
+### Run a Samza Job
+
+After you've built your Samza package, you can start a job on the grid using the run-job.sh script.
+
+    deploy/samza/bin/run-job.sh --config-factory=samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/wikipedia-feed.properties
+
+The job will consume a feed of real-time edits from Wikipedia, and produce them to a Kafka topic called "wikipedia-raw". Give the job a minute to startup, and then tail the Kafka topic:
+
+    deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181 --topic wikipedia-raw
+
+Pretty neat, right? Now, check out the YARN UI again ([http://localhost:8088](http://localhost:8088)). This time around, you'll see your Samza job is running!
+
+### Generate Wikipedia Statistics
+
+Let's calculate some statistics based on the messages in the wikipedia-raw topic. Start two more jobs:
+
+    deploy/samza/bin/run-job.sh --config-factory=samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/wikipedia-parser.properties
+    deploy/samza/bin/run-job.sh --config-factory=samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/wikipedia-stats.properties
+
+The first job (wikipedia-parser) parses the messages in wikipedia-raw, and extracts information about the size of the edit, who made the change, etc. You can take a look at its output with:
+
+    deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181 --topic wikipedia-edits
+
+The last job (wikipedia-stats) reads messages from the wikipedia-edits topic, and calculates counts, every ten seconds, for all edits that were made during that window. It outputs these counts to the wikipedia-stats topic.
+
+    deploy/kafka/bin/kafka-console-consumer.sh  --zookeeper localhost:2181 --topic wikipedia-stats
+
+The messages in the stats topic look like this:
+
+    {"is-talk":2,"bytes-added":5276,"edits":13,"unique-titles":13}
+    {"is-bot-edit":1,"is-talk":3,"bytes-added":4211,"edits":30,"unique-titles":30,"is-unpatrolled":1,"is-new":2,"is-minor":7}
+    {"bytes-added":3180,"edits":19,"unique-titles":19,"is-unpatrolled":1,"is-new":1,"is-minor":3}
+    {"bytes-added":2218,"edits":18,"unique-titles":18,"is-unpatrolled":2,"is-new":2,"is-minor":3}
+
+If you check the YARN UI, again, you'll see that all three jobs are now listed.
+
+### Shutdown
+
+After you're done, you can clean everything up using the same grid script.
+
+    bin/grid stop yarn
+    bin/grid stop kafka
+    bin/grid stop zookeeper
+
+Congratulations! You've now setup a local grid that includes YARN, Kafka, and ZooKeeper, and run a Samza job on it. Next up, check out the [Background](/learn/documentation/0.7.0/introduction/background.html) and [API Overview](/learn/documentation/0.7.0/api/overview.html) pages.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
new file mode 100644
index 0000000..f7730b1
--- /dev/null
+++ b/gradle.properties
@@ -0,0 +1,3 @@
+version=0.7.0
+scalaVersion=2.9.2
+yarnVersion=2.0.5-alpha

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/buildscript.gradle
----------------------------------------------------------------------
diff --git a/gradle/buildscript.gradle b/gradle/buildscript.gradle
new file mode 100644
index 0000000..225e0a8
--- /dev/null
+++ b/gradle/buildscript.gradle
@@ -0,0 +1,12 @@
+repositories {
+  repositories {
+    // For license plugin.
+    maven {
+      url 'http://dl.bintray.com/content/netflixoss/external-gradle-plugins/'
+    }
+  }
+}
+
+dependencies {
+  classpath 'nl.javadude.gradle.plugins:license-gradle-plugin:0.6.1'
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/dependency-versions-scala-2.8.1.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions-scala-2.8.1.gradle b/gradle/dependency-versions-scala-2.8.1.gradle
new file mode 100644
index 0000000..070c1ff
--- /dev/null
+++ b/gradle/dependency-versions-scala-2.8.1.gradle
@@ -0,0 +1,6 @@
+ext {
+  scalaVersion = "2.8.1"
+  grizzledVersion = "0.6.10"
+  scalatraVersion = "2.0.4"
+  jettyVersion = "7.0.0.v20091005"
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/dependency-versions-scala-2.9.2.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions-scala-2.9.2.gradle b/gradle/dependency-versions-scala-2.9.2.gradle
new file mode 100644
index 0000000..e1e574e
--- /dev/null
+++ b/gradle/dependency-versions-scala-2.9.2.gradle
@@ -0,0 +1,6 @@
+ext {
+  scalaVersion = "2.9.2"
+  grizzledVersion = "0.6.10"
+  scalatraVersion = "2.2.1"
+  jettyVersion = "8.1.8.v20121106"
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/dependency-versions.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle
new file mode 100644
index 0000000..5f3fb32
--- /dev/null
+++ b/gradle/dependency-versions.gradle
@@ -0,0 +1,12 @@
+ext {
+  joptSimpleVersion = "3.2"
+  jacksonVersion = "1.8.5"
+  junitVersion = "4.8.1"
+  mockitoVersion = "1.8.4"
+  zkClientVersion = "0.3"
+  zookeeperVersion = "3.3.4"
+  metricsVersion = "2.2.0"
+  kafkaVersion = "0.8.1-SNAPSHOT"
+  commonsHttpClientVersion = "3.1"
+  leveldbVersion = "1.7"
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/license.gradle
----------------------------------------------------------------------
diff --git a/gradle/license.gradle b/gradle/license.gradle
new file mode 100644
index 0000000..b4b62eb
--- /dev/null
+++ b/gradle/license.gradle
@@ -0,0 +1,9 @@
+subprojects {
+  apply plugin: 'license'
+
+  license {
+    header rootProject.file('HEADER')
+    // Skip Twitter bootstrap JS and CSS.
+    skipExistingHeaders = true
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/maven.gradle
----------------------------------------------------------------------
diff --git a/gradle/maven.gradle b/gradle/maven.gradle
new file mode 100644
index 0000000..3613879
--- /dev/null
+++ b/gradle/maven.gradle
@@ -0,0 +1,19 @@
+subprojects {
+  apply plugin: 'maven-publish'
+
+  publishing {
+    publications {
+      mavenJava(MavenPublication) {
+        from components.java
+
+        if(project.name == 'samza-shell') {
+          artifact shellTarGz {
+            group "org.apache.samza"
+            name "samza-shell"
+            classifier "dist"
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/wrapper/gradle-wrapper.jar
----------------------------------------------------------------------
diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar
new file mode 100644
index 0000000..a7634b0
Binary files /dev/null and b/gradle/wrapper/gradle-wrapper.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradle/wrapper/gradle-wrapper.properties
----------------------------------------------------------------------
diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
new file mode 100644
index 0000000..610282a
--- /dev/null
+++ b/gradle/wrapper/gradle-wrapper.properties
@@ -0,0 +1,6 @@
+#Thu Jul 11 22:18:11 PDT 2013
+distributionBase=GRADLE_USER_HOME
+distributionPath=wrapper/dists
+zipStoreBase=GRADLE_USER_HOME
+zipStorePath=wrapper/dists
+distributionUrl=http\://services.gradle.org/distributions/gradle-1.6-bin.zip

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/gradlew
----------------------------------------------------------------------
diff --git a/gradlew b/gradlew
new file mode 100755
index 0000000..16bbbbf
--- /dev/null
+++ b/gradlew
@@ -0,0 +1,164 @@
+#!/usr/bin/env bash
+
+##############################################################################
+##
+##  Gradle start up script for UN*X
+##
+##############################################################################
+
+# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
+DEFAULT_JVM_OPTS="-XX:MaxPermSize=512m"
+
+APP_NAME="Gradle"
+APP_BASE_NAME=`basename "$0"`
+
+# Use the maximum available, or set MAX_FD != -1 to use that value.
+MAX_FD="maximum"
+
+warn ( ) {
+    echo "$*"
+}
+
+die ( ) {
+    echo
+    echo "$*"
+    echo
+    exit 1
+}
+
+# OS specific support (must be 'true' or 'false').
+cygwin=false
+msys=false
+darwin=false
+case "`uname`" in
+  CYGWIN* )
+    cygwin=true
+    ;;
+  Darwin* )
+    darwin=true
+    ;;
+  MINGW* )
+    msys=true
+    ;;
+esac
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched.
+if $cygwin ; then
+    [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+fi
+
+# Attempt to set APP_HOME
+# Resolve links: $0 may be a link
+PRG="$0"
+# Need this for relative symlinks.
+while [ -h "$PRG" ] ; do
+    ls=`ls -ld "$PRG"`
+    link=`expr "$ls" : '.*-> \(.*\)$'`
+    if expr "$link" : '/.*' > /dev/null; then
+        PRG="$link"
+    else
+        PRG=`dirname "$PRG"`"/$link"
+    fi
+done
+SAVED="`pwd`"
+cd "`dirname \"$PRG\"`/" >&-
+APP_HOME="`pwd -P`"
+cd "$SAVED" >&-
+
+CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
+
+# Determine the Java command to use to start the JVM.
+if [ -n "$JAVA_HOME" ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+        # IBM's JDK on AIX uses strange locations for the executables
+        JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+        JAVACMD="$JAVA_HOME/bin/java"
+    fi
+    if [ ! -x "$JAVACMD" ] ; then
+        die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME
+
+Please set the JAVA_HOME variable in your environment to match the
+location of your Java installation."
+    fi
+else
+    JAVACMD="java"
+    which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
+
+Please set the JAVA_HOME variable in your environment to match the
+location of your Java installation."
+fi
+
+# Increase the maximum file descriptors if we can.
+if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then
+    MAX_FD_LIMIT=`ulimit -H -n`
+    if [ $? -eq 0 ] ; then
+        if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
+            MAX_FD="$MAX_FD_LIMIT"
+        fi
+        ulimit -n $MAX_FD
+        if [ $? -ne 0 ] ; then
+            warn "Could not set maximum file descriptor limit: $MAX_FD"
+        fi
+    else
+        warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT"
+    fi
+fi
+
+# For Darwin, add options to specify how the application appears in the dock
+if $darwin; then
+    GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\""
+fi
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin ; then
+    APP_HOME=`cygpath --path --mixed "$APP_HOME"`
+    CLASSPATH=`cygpath --path --mixed "$CLASSPATH"`
+
+    # We build the pattern for arguments to be converted via cygpath
+    ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null`
+    SEP=""
+    for dir in $ROOTDIRSRAW ; do
+        ROOTDIRS="$ROOTDIRS$SEP$dir"
+        SEP="|"
+    done
+    OURCYGPATTERN="(^($ROOTDIRS))"
+    # Add a user-defined pattern to the cygpath arguments
+    if [ "$GRADLE_CYGPATTERN" != "" ] ; then
+        OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)"
+    fi
+    # Now convert the arguments - kludge to limit ourselves to /bin/sh
+    i=0
+    for arg in "$@" ; do
+        CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -`
+        CHECK2=`echo "$arg"|egrep -c "^-"`                                 ### Determine if an option
+
+        if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then                    ### Added a condition
+            eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"`
+        else
+            eval `echo args$i`="\"$arg\""
+        fi
+        i=$((i+1))
+    done
+    case $i in
+        (0) set -- ;;
+        (1) set -- "$args0" ;;
+        (2) set -- "$args0" "$args1" ;;
+        (3) set -- "$args0" "$args1" "$args2" ;;
+        (4) set -- "$args0" "$args1" "$args2" "$args3" ;;
+        (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
+        (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
+        (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
+        (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
+        (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
+    esac
+fi
+
+# Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules
+function splitJvmOpts() {
+    JVM_OPTS=("$@")
+}
+eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS
+JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME"
+
+exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@"

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/Partition.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/Partition.java b/samza-api/src/main/java/org/apache/samza/Partition.java
new file mode 100644
index 0000000..d83a097
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/Partition.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza;
+
+public class Partition {
+  private final int partition;
+
+  public Partition(int partition) {
+    this.partition = partition;
+  }
+
+  public int getPartitionId() {
+    return partition;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + partition;
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Partition other = (Partition) obj;
+    if (partition != other.partition)
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "Partition [partition=" + partition + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/SamzaException.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/SamzaException.java b/samza-api/src/main/java/org/apache/samza/SamzaException.java
new file mode 100644
index 0000000..6b5a67b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/SamzaException.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza;
+
+/**
+ * Unchecked exception that Samza throws when something goes wrong.
+ */
+public class SamzaException extends RuntimeException {
+  private static final long serialVersionUID = 1L;
+
+  public SamzaException() {
+    super();
+  }
+
+  public SamzaException(String s, Throwable t) {
+    super(s, t);
+  }
+
+  public SamzaException(String s) {
+    super(s);
+  }
+
+  public SamzaException(Throwable t) {
+    super(t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java b/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java
new file mode 100644
index 0000000..22fa582
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint;
+
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.samza.system.SystemStream;
+
+public class Checkpoint {
+  private final Map<SystemStream, String> offsets;
+
+  public Checkpoint(Map<SystemStream, String> offsets) {
+    this.offsets = offsets;
+  }
+
+  public Map<SystemStream, String> getOffsets() {
+    return Collections.unmodifiableMap(offsets);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((offsets == null) ? 0 : offsets.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Checkpoint other = (Checkpoint) obj;
+    if (offsets == null) {
+      if (other.offsets != null)
+        return false;
+    } else if (!offsets.equals(other.offsets))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "Checkpoint [offsets=" + offsets + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
new file mode 100644
index 0000000..2890de5
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManager.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint;
+
+import org.apache.samza.Partition;
+
+public interface CheckpointManager {
+  public void start();
+
+  public void register(Partition partition);
+
+  public void writeCheckpoint(Partition partition, Checkpoint checkpoint);
+
+  public Checkpoint readLastCheckpoint(Partition partition);
+
+  public void stop();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManagerFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManagerFactory.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManagerFactory.java
new file mode 100644
index 0000000..5ce8f35
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointManagerFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.metrics.MetricsRegistry;
+
+public interface CheckpointManagerFactory {
+  public CheckpointManager getCheckpointManager(Config config, MetricsRegistry registry);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/config/Config.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/Config.java b/samza-api/src/main/java/org/apache/samza/config/Config.java
new file mode 100644
index 0000000..b2f8805
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/config/Config.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class Config implements Map<String, String> {
+  public Config subset(String prefix) {
+    return subset(prefix, true);
+  }
+
+  public Config subset(String prefix, boolean stripPrefix) {
+    Map<String, String> out = new HashMap<String, String>();
+
+    for (Entry<String, String> entry : entrySet()) {
+      String k = entry.getKey();
+      if (k != null && k.startsWith(prefix)) {
+        if (stripPrefix) {
+          k = k.substring(prefix.length());
+        }
+
+        out.put(k, entry.getValue());
+      }
+    }
+
+    return new MapConfig(out);
+  }
+
+  public String get(String k, String defaultString) {
+    if (!containsKey(k)) {
+      return defaultString;
+    }
+    return get(k);
+  }
+
+  public boolean getBoolean(String k, boolean defaultValue) {
+    if (containsKey(k))
+      return "true".equalsIgnoreCase(get(k));
+    else
+      return defaultValue;
+  }
+
+  public boolean getBoolean(String k) {
+    if (containsKey(k))
+      return "true".equalsIgnoreCase(get(k));
+    else
+      throw new ConfigException("Missing key " + k + ".");
+  }
+
+  public long getShort(String k, short defaultValue) {
+    if (containsKey(k))
+      return Short.parseShort(get(k));
+    else
+      return defaultValue;
+  }
+
+  public long getShort(String k) {
+    if (containsKey(k))
+      return Short.parseShort(get(k));
+    else
+      throw new ConfigException("Missing key " + k + ".");
+  }
+
+  public long getLong(String k, long defaultValue) {
+    if (containsKey(k))
+      return Long.parseLong(get(k));
+    else
+      return defaultValue;
+  }
+
+  public long getLong(String k) {
+    if (containsKey(k))
+      return Long.parseLong(get(k));
+    else
+      throw new ConfigException("Missing key " + k + ".");
+  }
+
+  public int getInt(String k, int defaultValue) {
+    if (containsKey(k))
+      return Integer.parseInt(get(k));
+    else
+      return defaultValue;
+  }
+
+  public int getInt(String k) {
+    if (containsKey(k))
+      return Integer.parseInt(get(k));
+    else
+      throw new ConfigException("Missing key " + k + ".");
+  }
+
+  public double getDouble(String k, double defaultValue) {
+    if (containsKey(k))
+      return Double.parseDouble(get(k));
+    else
+      return defaultValue;
+  }
+
+  public double getDouble(String k) {
+    if (containsKey(k))
+      return Double.parseDouble(get(k));
+    else
+      throw new ConfigException("Missing key " + k + ".");
+  }
+
+  public List<String> getList(String k, List<String> defaultValue) {
+    if (!containsKey(k))
+      return defaultValue;
+
+    String value = get(k);
+    String[] pieces = value.split("\\s*,\\s*");
+    return Arrays.asList(pieces);
+  }
+
+  public List<String> getList(String k) {
+    if (!containsKey(k))
+      throw new ConfigException("Missing key " + k + ".");
+    return getList(k, null);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> Class<T> getClass(String k) {
+    if (containsKey(k)) {
+      try {
+        return (Class<T>) Class.forName(get(k));
+      } catch (Exception e) {
+        throw new ConfigException("Unable to find class.", e);
+      }
+    } else {
+      throw new ConfigException("Missing key " + k + ".");
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> T getNewInstance(String k) {
+    try {
+      return (T) getClass(k).newInstance();
+    } catch (Exception e) {
+      throw new ConfigException("Unable to instantiate class.", e);
+    }
+  }
+
+  public Date getDate(String k) {
+    return getDate(k, new SimpleDateFormat());
+  }
+
+  public Date getDate(String k, String format) {
+    return getDate(k, new SimpleDateFormat(format));
+  }
+
+  public Date getDate(String k, SimpleDateFormat format) {
+    if (!containsKey(k))
+      throw new ConfigException("Missing key " + k + ".");
+
+    try {
+      return format.parse(get(k));
+    } catch (ParseException e) {
+      throw new ConfigException("Date format exception.", e);
+    }
+  }
+
+  public Date getDate(String k, Date defaultValue) {
+    return getDate(k, new SimpleDateFormat(), defaultValue);
+  }
+
+  public Date getDate(String k, String format, Date defaultValue) {
+    return getDate(k, new SimpleDateFormat(format), defaultValue);
+  }
+
+  public Date getDate(String k, SimpleDateFormat format, Date defaultValue) {
+    if (!containsKey(k))
+      return defaultValue;
+
+    try {
+      return format.parse(get(k));
+    } catch (ParseException e) {
+      throw new ConfigException("Date format exception.", e);
+    }
+  }
+
+  public void clear() {
+    throw new ConfigException("Config is immutable.");
+  }
+
+  public String put(String key, String value) {
+    throw new ConfigException("Config is immutable.");
+  }
+
+  public void putAll(Map<? extends String, ? extends String> m) {
+    throw new ConfigException("Config is immutable.");
+  }
+
+  public String remove(Object s) {
+    throw new ConfigException("Config is immutable.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/config/ConfigException.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigException.java b/samza-api/src/main/java/org/apache/samza/config/ConfigException.java
new file mode 100644
index 0000000..b6ab549
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/config/ConfigException.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config;
+
+import org.apache.samza.SamzaException;
+
+public class ConfigException extends SamzaException {
+  private static final long serialVersionUID = 1L;
+
+  public ConfigException(Throwable e) {
+    super(e);
+  }
+
+  public ConfigException(String msg) {
+    super(msg);
+  }
+
+  public ConfigException(String msg, Throwable e) {
+    super(msg, e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/config/ConfigFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigFactory.java b/samza-api/src/main/java/org/apache/samza/config/ConfigFactory.java
new file mode 100644
index 0000000..d6d7584
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/config/ConfigFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config;
+
+import java.net.URI;
+
+public interface ConfigFactory {
+  Config getConfig(URI configUri);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/config/ConfigRewriter.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigRewriter.java b/samza-api/src/main/java/org/apache/samza/config/ConfigRewriter.java
new file mode 100644
index 0000000..7248e8b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/config/ConfigRewriter.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config;
+
+/**
+ * Re-write the job's config before the job is submitted.
+ */
+public interface ConfigRewriter {
+  Config rewrite(String name, Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
new file mode 100644
index 0000000..337e921
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class MapConfig extends Config {
+  private final Map<String, String> map;
+  
+  public MapConfig() {
+    this.map = Collections.emptyMap();
+  }
+  
+  public MapConfig(Map<String, String> map) {
+    this(Collections.singletonList(map));
+  }
+
+  public MapConfig(List<Map<String, String>> maps) {
+    this.map = new HashMap<String, String>();
+    for(Map<String, String> m: maps)
+      this.map.putAll(m);
+  }
+
+  public String get(Object k) {
+    return map.get(k);
+  }
+
+  public boolean containsKey(Object k) {
+    return map.containsKey(k);
+  }
+
+  public Set<Map.Entry<String, String>> entrySet() {
+    return map.entrySet();
+  }
+
+  public boolean isEmpty() {
+    return map.isEmpty();
+  }
+
+  public Set<String> keySet() {
+    return map.keySet();
+  }
+
+  public int size() {
+    return map.size();
+  }
+
+  public Collection<String> values() {
+    return map.values();
+  }
+
+  public boolean containsValue(Object v) {
+    return map.containsKey(v);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((map == null) ? 0 : map.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    MapConfig other = (MapConfig) obj;
+    if (map == null) {
+      if (other.map != null)
+        return false;
+    } else if (!map.equals(other.map))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return map.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/job/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/job/ApplicationStatus.java b/samza-api/src/main/java/org/apache/samza/job/ApplicationStatus.java
new file mode 100644
index 0000000..49052af
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/job/ApplicationStatus.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job;
+
+public enum ApplicationStatus {
+  Running("Running"), SuccessfulFinish("SuccessfulFinish"), UnsuccessfulFinish("UnsuccessfulFinish"), New("New");
+
+  private final String str;
+
+  private ApplicationStatus(String str) {
+    this.str = str;
+  }
+
+  public String toString() {
+    return str;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/job/CommandBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/job/CommandBuilder.java b/samza-api/src/main/java/org/apache/samza/job/CommandBuilder.java
new file mode 100644
index 0000000..934423b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/job/CommandBuilder.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+
+public abstract class CommandBuilder {
+  protected Set<Partition> partitions;
+  protected int totalPartitions;
+  protected String name;
+  protected Config config;
+
+  public CommandBuilder setPartitions(Set<Partition> partitions) {
+    this.partitions = partitions;
+    return this;
+  }
+
+  /**
+   * @param name
+   *          associated with a specific instantiation of a TaskRunner.
+   * @return self to support a builder style of use.
+   */
+  public CommandBuilder setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public CommandBuilder setConfig(Config config) {
+    this.config = config;
+    return this;
+  }
+
+  public CommandBuilder setTotalPartitions(int totalPartitions) {
+    this.totalPartitions = totalPartitions;
+    return this;
+  }
+
+  public abstract String buildCommand();
+
+  public abstract Map<String, String> buildEnvironment();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/job/StreamJob.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/job/StreamJob.java b/samza-api/src/main/java/org/apache/samza/job/StreamJob.java
new file mode 100644
index 0000000..f519949
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/job/StreamJob.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job;
+
+public interface StreamJob {
+  StreamJob submit();
+
+  StreamJob kill();
+
+  ApplicationStatus waitForFinish(long timeoutMs);
+
+  ApplicationStatus waitForStatus(ApplicationStatus status, long timeoutMs);
+
+  ApplicationStatus getStatus();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/job/StreamJobFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/job/StreamJobFactory.java b/samza-api/src/main/java/org/apache/samza/job/StreamJobFactory.java
new file mode 100644
index 0000000..4cdcc2c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/job/StreamJobFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job;
+
+import org.apache.samza.config.Config;
+
+public interface StreamJobFactory {
+  StreamJob getJob(Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/Counter.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/Counter.java b/samza-api/src/main/java/org/apache/samza/metrics/Counter.java
new file mode 100644
index 0000000..0838df3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/Counter.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A counter is a metric that represents a cumulative value.
+ */
+public class Counter implements Metric {
+  private final String name;
+  private final AtomicLong count;
+
+  public Counter(String name) {
+    this.name = name;
+    this.count = new AtomicLong(0);
+  }
+
+  public long inc() {
+    return inc(1);
+  }
+
+  public long inc(long n) {
+    return count.addAndGet(n);
+  }
+
+  public long dec() {
+    return dec(1);
+  }
+
+  public long dec(long n) {
+    return count.addAndGet(0 - n);
+  }
+  
+  public void set(long n) {
+    count.set(n);
+  }
+
+  public void clear() {
+    count.set(0);
+  }
+
+  public long getCount() {
+    return count.get();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void visit(MetricsVisitor visitor) {
+    visitor.counter(this);
+  }
+
+  public String toString() {
+    return count.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/DataType.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DataType.java b/samza-api/src/main/java/org/apache/samza/metrics/DataType.java
new file mode 100644
index 0000000..dc490d4
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/DataType.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+public enum DataType {
+  DataLong("DataLong"), DataDouble("DataDouble"), DataString("DataString");
+
+  private final String str;
+
+  private DataType(String str) {
+    this.str = str;
+  }
+
+  public String toString() {
+    return str;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/Gauge.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/Gauge.java b/samza-api/src/main/java/org/apache/samza/metrics/Gauge.java
new file mode 100644
index 0000000..14db2d3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/Gauge.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+public class Gauge<T> implements Metric {
+  private final String name;
+  private AtomicReference<T> ref;
+
+  public Gauge(String name, T value) {
+    this.name = name;
+    this.ref = new AtomicReference<T>(value);
+  }
+
+  public T set(T n) {
+    return ref.getAndSet(n);
+  }
+
+  public T getValue() {
+    return ref.get();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void visit(MetricsVisitor visitor) {
+    visitor.gauge(this);
+  }
+
+  public String toString() {
+    T value = ref.get();
+    return (value == null) ? null : value.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/Metric.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/Metric.java b/samza-api/src/main/java/org/apache/samza/metrics/Metric.java
new file mode 100644
index 0000000..a13e896
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/Metric.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+/**
+ * Metric class that allows metric visitors to visit it to get its information.
+ */
+public interface Metric {
+  void visit(MetricsVisitor visitor);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java
new file mode 100644
index 0000000..a4563c4
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+public interface MetricsRegistry {
+  Counter newCounter(String group, String name);
+
+  <T> Gauge<T> newGauge(String group, String name, T value);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporter.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporter.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporter.java
new file mode 100644
index 0000000..d52dfa9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporter.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+public interface MetricsReporter {
+  void start();
+
+  void register(String source, ReadableMetricsRegistry registry);
+
+  void stop();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java
new file mode 100644
index 0000000..19eb91c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsReporterFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+import org.apache.samza.config.Config;
+
+public interface MetricsReporterFactory {
+  MetricsReporter getMetricsReporter(String name, String containerName, Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/MetricsType.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsType.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsType.java
new file mode 100644
index 0000000..e79d4e6
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsType.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+public enum MetricsType {
+  MetricsCounter("MetricsCounter"), MetricsGauge("MetricsGauge");
+
+  private final String str;
+
+  private MetricsType(String str) {
+    this.str = str;
+  }
+
+  public String toString() {
+    return str;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java
new file mode 100644
index 0000000..fee0883
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+/**
+ * A metric visitor visits a metric, before metrics are flushed to a metrics stream.
+ */
+public abstract class MetricsVisitor {
+  public abstract void counter(Counter counter);
+
+  public abstract <T> void gauge(Gauge<T> gauge);
+
+  public void visit(Metric metric) {
+    if (metric instanceof Counter) {
+      counter((Counter) metric);
+    } else if (metric instanceof Gauge<?>) {
+      gauge((Gauge<?>) metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistry.java
new file mode 100644
index 0000000..ebea426
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistry.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+import java.util.Map;
+import java.util.Set;
+
+public interface ReadableMetricsRegistry extends MetricsRegistry {
+  Set<String> getGroups();
+
+  Map<String, Metric> getGroup(String group);
+
+  void listen(ReadableMetricsRegistryListener listener);
+
+  void unlisten(ReadableMetricsRegistryListener listener);
+}


[13/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/job_detail.graffle
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/job_detail.graffle b/docs/img/0.7.0/learn/documentation/introduction/job_detail.graffle
new file mode 100644
index 0000000..1583d55
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/introduction/job_detail.graffle
@@ -0,0 +1,1320 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN" "http://www.apple.com/DTDs/PropertyList-1.0.dtd">
+<plist version="1.0">
+<dict>
+	<key>ActiveLayerIndex</key>
+	<integer>0</integer>
+	<key>ApplicationVersion</key>
+	<array>
+		<string>com.omnigroup.OmniGrafflePro.MacAppStore</string>
+		<string>139.18</string>
+	</array>
+	<key>AutoAdjust</key>
+	<true/>
+	<key>BackgroundGraphic</key>
+	<dict>
+		<key>Bounds</key>
+		<string>{{0, 0}, {576.00002479553223, 733}}</string>
+		<key>Class</key>
+		<string>SolidGraphic</string>
+		<key>ID</key>
+		<integer>2</integer>
+		<key>Style</key>
+		<dict>
+			<key>shadow</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+			<key>stroke</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+		</dict>
+	</dict>
+	<key>BaseZoom</key>
+	<integer>0</integer>
+	<key>CanvasOrigin</key>
+	<string>{0, 0}</string>
+	<key>ColumnAlign</key>
+	<integer>1</integer>
+	<key>ColumnSpacing</key>
+	<real>36</real>
+	<key>CreationDate</key>
+	<string>2013-07-28 22:34:27 +0000</string>
+	<key>Creator</key>
+	<string>Jay Kreps</string>
+	<key>DisplayScale</key>
+	<string>1 0/72 in = 1 0/72 in</string>
+	<key>GraphDocumentVersion</key>
+	<integer>8</integer>
+	<key>GraphicsList</key>
+	<array>
+		<dict>
+			<key>Bounds</key>
+			<string>{{59.583091735839844, 17}, {71, 17}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>59</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs28 \cf0 Samza Job}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{101.5, 113.67618703842163}, {53, 34}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>38</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{-0.59628479784302701, -1.1925696134567261}</string>
+				<string>{1.9868216701487083e-08, -1.3333333730697632}</string>
+				<string>{0.59628487781105233, -1.1925696134567261}</string>
+				<string>{1.1925696134567272, -0.59628480672836304}</string>
+				<string>{1.3333333730697643, 1.5894572413799324e-07}</string>
+				<string>{1.1925696134567272, 0.59628473564567486}</string>
+				<string>{0.59628465308492307, 1.1925697326660156}</string>
+				<string>{1.1842379282265398e-15, 1.3333333730697632}</string>
+				<string>{-0.5962849488937394, 1.1925696134567261}</string>
+				<string>{-1.1925697326660152, 0.5962844398368361}</string>
+				<string>{-1.3333333730697625, -6.3578289655197295e-07}</string>
+				<string>{-1.1925696134567256, -0.59628480672836304}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 Task 2}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{126.51412229515523, 156.70258726469865}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>53</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>51</integer>
+				<key>Position</key>
+				<real>0.37774357199668884</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{102.02543265311756, 155.54622220993042}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>52</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>50</integer>
+				<key>Position</key>
+				<real>0.30659866333007812</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>51</integer>
+			<key>Points</key>
+			<array>
+				<string>{142.95187377929688, 143.38228750228882}</string>
+				<string>{110.02543265311758, 197.17618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>50</integer>
+			<key>Points</key>
+			<array>
+				<string>{114.74999684095383, 147.67618703842163}</string>
+				<string>{86.294020898085591, 196.17618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>38</integer>
+				<key>Info</key>
+				<integer>9</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{83.310078640505097, 156.09778326749802}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>49</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>47</integer>
+				<key>Position</key>
+				<real>0.31154739856719971</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{59.583090066240111, 154.42674145102501}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>48</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>32</integer>
+				<key>Position</key>
+				<real>0.28351658582687378</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>47</integer>
+			<key>Points</key>
+			<array>
+				<string>{78.249995261430769, 147.67618703842163}</string>
+				<string>{107.33091014225297, 197.17618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>37</integer>
+				<key>Info</key>
+				<integer>7</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{74.999992370605469, 197.17618703842163}, {46, 26}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>46</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 Output \
+Stream C}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{137.96903325656112, 78.093316394752037}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>45</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>42</integer>
+				<key>Position</key>
+				<real>0.4247739315032959</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{106.55560185205636, 77.46781401806038}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>44</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>41</integer>
+				<key>Position</key>
+				<real>0.41218578815460205</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{115.5, 41.796580791473389}, {46, 26}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>43</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 Input \
+Stream B}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>38</integer>
+				<key>Info</key>
+				<integer>3</integer>
+			</dict>
+			<key>ID</key>
+			<integer>42</integer>
+			<key>Points</key>
+			<array>
+				<string>{142.5, 63.986382961273193}</string>
+				<string>{141.25000157952311, 113.67618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>37</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>41</integer>
+			<key>Points</key>
+			<array>
+				<string>{142.5, 63.986382961273193}</string>
+				<string>{65.000001053015481, 113.67618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>42</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{72.487797844925495, 76.103014686803647}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>40</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>22</integer>
+				<key>Position</key>
+				<real>0.39506399631500244</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 1}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{47.596244023754139, 75.593161679834566}, {8, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica-Light</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>39</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>21</integer>
+				<key>Position</key>
+				<real>0.38497579097747803</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 0}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{38.5, 113.67618703842163}, {53, 34}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>37</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{-0.59628479784302701, -1.1925696134567261}</string>
+				<string>{1.9868216701487083e-08, -1.3333333730697632}</string>
+				<string>{0.59628487781105233, -1.1925696134567261}</string>
+				<string>{1.1925696134567272, -0.59628480672836304}</string>
+				<string>{1.3333333730697643, 1.5894572413799324e-07}</string>
+				<string>{1.1925696134567272, 0.59628473564567486}</string>
+				<string>{0.59628465308492307, 1.1925697326660156}</string>
+				<string>{1.1842379282265398e-15, 1.3333333730697632}</string>
+				<string>{-0.5962849488937394, 1.1925696134567261}</string>
+				<string>{-1.1925697326660152, 0.5962844398368361}</string>
+				<string>{-1.3333333730697625, -6.3578289655197295e-07}</string>
+				<string>{-1.1925696134567256, -0.59628480672836304}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 Task 1}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{28.596244812011719, 41.796580791473389}, {46, 26}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>35</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica-Light;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs22 \cf0 Input \
+Stream A}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>50</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+			<key>ID</key>
+			<integer>32</integer>
+			<key>Points</key>
+			<array>
+				<string>{54.596244812011719, 147.67618703842163}</string>
+				<string>{86.294020898085591, 196.17618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>38</integer>
+			</dict>
+			<key>ID</key>
+			<integer>22</integer>
+			<key>Points</key>
+			<array>
+				<string>{51.5, 63.136671841144562}</string>
+				<string>{114.75000019744039, 113.67618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>21</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>37</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+			<key>ID</key>
+			<integer>21</integer>
+			<key>Points</key>
+			<array>
+				<string>{51.5, 63.136671841144562}</string>
+				<string>{51.750000197440386, 113.67618703842163}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>StickArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{25, 96.176187038421631}, {136.5, 67}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>19</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{-0.59628479784302701, -1.1925696134567261}</string>
+				<string>{1.9868216701487083e-08, -1.3333333730697632}</string>
+				<string>{0.59628487781105233, -1.1925696134567261}</string>
+				<string>{1.1925696134567272, -0.59628480672836304}</string>
+				<string>{1.3333333730697643, 1.5894572413799324e-07}</string>
+				<string>{1.1925696134567272, 0.59628473564567486}</string>
+				<string>{0.59628465308492307, 1.1925697326660156}</string>
+				<string>{1.1842379282265398e-15, 1.3333333730697632}</string>
+				<string>{-0.5962849488937394, 1.1925696134567261}</string>
+				<string>{-1.1925697326660152, 0.5962844398368361}</string>
+				<string>{-1.3333333730697625, -6.3578289655197295e-07}</string>
+				<string>{-1.1925696134567256, -0.59628480672836304}</string>
+			</array>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Pattern</key>
+					<integer>1</integer>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+	</array>
+	<key>GridInfo</key>
+	<dict/>
+	<key>GuidesLocked</key>
+	<string>NO</string>
+	<key>GuidesVisible</key>
+	<string>YES</string>
+	<key>HPages</key>
+	<integer>1</integer>
+	<key>ImageCounter</key>
+	<integer>1</integer>
+	<key>KeepToScale</key>
+	<false/>
+	<key>Layers</key>
+	<array>
+		<dict>
+			<key>Lock</key>
+			<string>NO</string>
+			<key>Name</key>
+			<string>Layer 1</string>
+			<key>Print</key>
+			<string>YES</string>
+			<key>View</key>
+			<string>YES</string>
+		</dict>
+	</array>
+	<key>LayoutInfo</key>
+	<dict>
+		<key>Animate</key>
+		<string>NO</string>
+		<key>circoMinDist</key>
+		<real>18</real>
+		<key>circoSeparation</key>
+		<real>0.0</real>
+		<key>layoutEngine</key>
+		<string>dot</string>
+		<key>neatoSeparation</key>
+		<real>0.0</real>
+		<key>twopiSeparation</key>
+		<real>0.0</real>
+	</dict>
+	<key>LinksVisible</key>
+	<string>NO</string>
+	<key>MagnetsVisible</key>
+	<string>NO</string>
+	<key>MasterSheets</key>
+	<array/>
+	<key>ModificationDate</key>
+	<string>2013-07-28 22:49:23 +0000</string>
+	<key>Modifier</key>
+	<string>Jay Kreps</string>
+	<key>NotesVisible</key>
+	<string>NO</string>
+	<key>Orientation</key>
+	<integer>2</integer>
+	<key>OriginVisible</key>
+	<string>NO</string>
+	<key>PageBreaks</key>
+	<string>YES</string>
+	<key>PrintInfo</key>
+	<dict>
+		<key>NSBottomMargin</key>
+		<array>
+			<string>float</string>
+			<string>41</string>
+		</array>
+		<key>NSHorizonalPagination</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAhOU051bWJlcgCEhAdOU1ZhbHVlAISECE5TT2JqZWN0AIWEASqEhAFxlwCG</string>
+		</array>
+		<key>NSLeftMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSPaperSize</key>
+		<array>
+			<string>size</string>
+			<string>{612.00002479553223, 792}</string>
+		</array>
+		<key>NSPrintReverseOrientation</key>
+		<array>
+			<string>int</string>
+			<string>0</string>
+		</array>
+		<key>NSRightMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSTopMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+	</dict>
+	<key>PrintOnePage</key>
+	<false/>
+	<key>ReadOnly</key>
+	<string>NO</string>
+	<key>RowAlign</key>
+	<integer>1</integer>
+	<key>RowSpacing</key>
+	<real>36</real>
+	<key>SheetTitle</key>
+	<string>Canvas 1</string>
+	<key>SmartAlignmentGuidesActive</key>
+	<string>YES</string>
+	<key>SmartDistanceGuidesActive</key>
+	<string>YES</string>
+	<key>UniqueID</key>
+	<integer>1</integer>
+	<key>UseEntirePage</key>
+	<false/>
+	<key>VPages</key>
+	<integer>1</integer>
+	<key>WindowInfo</key>
+	<dict>
+		<key>CurrentSheet</key>
+		<integer>0</integer>
+		<key>ExpandedCanvases</key>
+		<array>
+			<dict>
+				<key>name</key>
+				<string>Canvas 1</string>
+			</dict>
+		</array>
+		<key>Frame</key>
+		<string>{{706, 6}, {711, 872}}</string>
+		<key>ListView</key>
+		<true/>
+		<key>OutlineWidth</key>
+		<integer>142</integer>
+		<key>RightSidebar</key>
+		<false/>
+		<key>ShowRuler</key>
+		<true/>
+		<key>Sidebar</key>
+		<true/>
+		<key>SidebarWidth</key>
+		<integer>120</integer>
+		<key>VisibleRegion</key>
+		<string>{{0, 0}, {576, 733}}</string>
+		<key>Zoom</key>
+		<real>1</real>
+		<key>ZoomValues</key>
+		<array>
+			<array>
+				<string>Canvas 1</string>
+				<real>1</real>
+				<real>1</real>
+			</array>
+		</array>
+	</dict>
+</dict>
+</plist>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/job_detail.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/job_detail.png b/docs/img/0.7.0/learn/documentation/introduction/job_detail.png
new file mode 100644
index 0000000..31f6707
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/job_detail.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza-ecosystem.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza-ecosystem.png b/docs/img/0.7.0/learn/documentation/introduction/samza-ecosystem.png
new file mode 100644
index 0000000..4eb9f33
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/samza-ecosystem.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza-hadoop.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza-hadoop.png b/docs/img/0.7.0/learn/documentation/introduction/samza-hadoop.png
new file mode 100644
index 0000000..2142f94
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/samza-hadoop.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png b/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png
new file mode 100644
index 0000000..2748713
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png b/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png
new file mode 100644
index 0000000..c897499
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza_state.graffle
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza_state.graffle b/docs/img/0.7.0/learn/documentation/introduction/samza_state.graffle
new file mode 100644
index 0000000..2427ce3
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/introduction/samza_state.graffle
@@ -0,0 +1,1654 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE plist PUBLIC "-//Apple//DTD PLIST 1.0//EN" "http://www.apple.com/DTDs/PropertyList-1.0.dtd">
+<plist version="1.0">
+<dict>
+	<key>ActiveLayerIndex</key>
+	<integer>0</integer>
+	<key>ApplicationVersion</key>
+	<array>
+		<string>com.omnigroup.OmniGrafflePro.MacAppStore</string>
+		<string>139.18</string>
+	</array>
+	<key>AutoAdjust</key>
+	<true/>
+	<key>BackgroundGraphic</key>
+	<dict>
+		<key>Bounds</key>
+		<string>{{0, 0}, {576.00002479553223, 733}}</string>
+		<key>Class</key>
+		<string>SolidGraphic</string>
+		<key>ID</key>
+		<integer>2</integer>
+		<key>Style</key>
+		<dict>
+			<key>shadow</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+			<key>stroke</key>
+			<dict>
+				<key>Draws</key>
+				<string>NO</string>
+			</dict>
+		</dict>
+	</dict>
+	<key>BaseZoom</key>
+	<integer>0</integer>
+	<key>CanvasOrigin</key>
+	<string>{0, 0}</string>
+	<key>ColumnAlign</key>
+	<integer>1</integer>
+	<key>ColumnSpacing</key>
+	<real>36</real>
+	<key>CreationDate</key>
+	<string>2013-07-24 20:59:23 +0000</string>
+	<key>Creator</key>
+	<string>Jay Kreps</string>
+	<key>DisplayScale</key>
+	<string>1 0/72 in = 1 0/72 in</string>
+	<key>GraphDocumentVersion</key>
+	<integer>8</integer>
+	<key>GraphicsList</key>
+	<array>
+		<dict>
+			<key>Bounds</key>
+			<string>{{354.00000274926424, 96.386200297623873}, {56, 16}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>50</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>2</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 DB Queries}</string>
+				<key>VerticalPad</key>
+				<integer>2</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>32</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>47</integer>
+			<key>Points</key>
+			<array>
+				<string>{375.94502185497402, 143.5769404965329}</string>
+				<string>{400.50000206194818, 131}</string>
+				<string>{428.74999985404742, 131}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>27</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>32</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>46</integer>
+			<key>Points</key>
+			<array>
+				<string>{317.00000015459955, 138}</string>
+				<string>{351.50000206194818, 118}</string>
+				<string>{384.50000206194818, 118}</string>
+				<string>{429.25000015459955, 131}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>29</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>32</integer>
+				<key>Info</key>
+				<integer>2</integer>
+			</dict>
+			<key>ID</key>
+			<integer>45</integer>
+			<key>Points</key>
+			<array>
+				<string>{274.95482065734831, 134.82683104667493}</string>
+				<string>{320.50000206194818, 106}</string>
+				<string>{382.50000206194818, 108}</string>
+				<string>{429.25000015459955, 131}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>28</integer>
+				<key>Position</key>
+				<real>0.96286928653717041</real>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{282.7798816413071, 176.46626249661199}, {70, 16}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>44</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>41</integer>
+				<key>Position</key>
+				<real>0.41301777958869934</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>2</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Output Stream}</string>
+				<key>VerticalPad</key>
+				<integer>2</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{265.50000206194818, 145.99999260902405}, {104, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>43</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Stateless Samza Tasks}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>42</integer>
+			<key>Points</key>
+			<array>
+				<string>{361.63447776995599, 166}</string>
+				<string>{362.63447776995599, 211}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>41</integer>
+			<key>Points</key>
+			<array>
+				<string>{317.62499995563974, 166.49998140404247}</string>
+				<string>{318, 210}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>26</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>40</integer>
+			<key>Points</key>
+			<array>
+				<string>{277.25000206194818, 166}</string>
+				<string>{278.25000206194818, 210}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{408.75000015459955, 131}, {41, 35}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>32</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Cylinder</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 DB}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{241.50000274926424, 49.500000346451998}, {62, 16}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>31</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>2</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Input Stream}</string>
+				<key>VerticalPad</key>
+				<integer>2</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>30</integer>
+			<key>Points</key>
+			<array>
+				<string>{281.09627582033107, 65.500000346451998}</string>
+				<string>{359.00000015459955, 138}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>31</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>29</integer>
+			<key>Points</key>
+			<array>
+				<string>{276.92236275888934, 65.500000346451998}</string>
+				<string>{317.00000015459955, 138}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>31</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>25</integer>
+			</dict>
+			<key>ID</key>
+			<integer>28</integer>
+			<key>Points</key>
+			<array>
+				<string>{272.75397078450942, 65.500000346451984}</string>
+				<string>{275.03969108280035, 137.50025175082229}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>31</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{343.50000015459955, 138}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>27</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{301.50000015459955, 138}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>26</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{259.50000015459955, 138}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>25</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{59.223656370973515, 185.64922458224424}, {70, 16}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>24</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>22</integer>
+				<key>Position</key>
+				<real>0.36017578840255737</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>2</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Output Stream}</string>
+				<key>VerticalPad</key>
+				<integer>2</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{46.00000206194818, 167.9414598941803}, {96, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>17</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Stateful Samza Tasks}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>23</integer>
+			<key>Points</key>
+			<array>
+				<string>{134.13447776995599, 171}</string>
+				<string>{135.13447776995599, 216}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>22</integer>
+			<key>Points</key>
+			<array>
+				<string>{94.068093287685457, 179.9414598941803}</string>
+				<string>{94.50000206194818, 218}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>17</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>21</integer>
+			<key>Points</key>
+			<array>
+				<string>{49.75000206194818, 171}</string>
+				<string>{49.75000206194818, 221}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{130.75000206194818, 151}, {11.5, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>20</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Cylinder</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{88.75000206194818, 151}, {11.5, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>48</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Cylinder</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{49.75000206194818, 151}, {11.5, 12}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>49</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Cylinder</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{110.13448295556009, 98.386206388473497}, {49, 14}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>16</integer>
+			<key>Line</key>
+			<dict>
+				<key>ID</key>
+				<integer>15</integer>
+				<key>Position</key>
+				<real>0.37310343980789185</real>
+				<key>RotationType</key>
+				<integer>0</integer>
+			</dict>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>1</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 changelog}</string>
+				<key>VerticalPad</key>
+				<integer>1</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>10</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+			<key>ID</key>
+			<integer>15</integer>
+			<key>Points</key>
+			<array>
+				<string>{136.50000015459955, 82.999999999999986}</string>
+				<string>{131.50000015459955, 143}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>12</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>9</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+			<key>ID</key>
+			<integer>14</integer>
+			<key>Points</key>
+			<array>
+				<string>{136.19166917030824, 83.393614022499548}</string>
+				<string>{89.500000154599547, 143}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>12</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>5</integer>
+			</dict>
+			<key>ID</key>
+			<integer>13</integer>
+			<key>Points</key>
+			<array>
+				<string>{136.11642027561359, 83.320733458135152}</string>
+				<string>{64.383577368540188, 143.30073769681798}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>12</integer>
+				<key>Info</key>
+				<integer>1</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{116.00000015459955, 48}, {41, 35}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>12</integer>
+			<key>Magnets</key>
+			<array>
+				<string>{0, 1}</string>
+				<string>{0, -1}</string>
+				<string>{1, 0}</string>
+				<string>{-1, 0}</string>
+			</array>
+			<key>Shape</key>
+			<string>Cylinder</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\qc
+
+\f0\fs20 \cf0 DB}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{14.00000274926424, 54.500000346451998}, {62, 16}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Color</key>
+				<dict>
+					<key>w</key>
+					<string>0</string>
+				</dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>10</real>
+			</dict>
+			<key>ID</key>
+			<integer>11</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>2</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs20 \cf0 Input Stream}</string>
+				<key>VerticalPad</key>
+				<integer>2</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>10</integer>
+			<key>Points</key>
+			<array>
+				<string>{53.596275820331059, 70.500000346451998}</string>
+				<string>{131.50000015459955, 143}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>11</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>ID</key>
+			<integer>9</integer>
+			<key>Points</key>
+			<array>
+				<string>{49.422362758889307, 70.500000346451998}</string>
+				<string>{89.500000154599547, 143}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>11</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Class</key>
+			<string>LineGraphic</string>
+			<key>Head</key>
+			<dict>
+				<key>ID</key>
+				<integer>5</integer>
+			</dict>
+			<key>ID</key>
+			<integer>8</integer>
+			<key>Points</key>
+			<array>
+				<string>{45.253970784509427, 70.500000346451984}</string>
+				<string>{47.53969108280036, 142.50025175082229}</string>
+			</array>
+			<key>Style</key>
+			<dict>
+				<key>stroke</key>
+				<dict>
+					<key>HeadArrow</key>
+					<string>FilledArrow</string>
+					<key>Legacy</key>
+					<true/>
+					<key>LineType</key>
+					<integer>1</integer>
+					<key>TailArrow</key>
+					<string>0</string>
+				</dict>
+			</dict>
+			<key>Tail</key>
+			<dict>
+				<key>ID</key>
+				<integer>11</integer>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{116.00000015459955, 143}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>7</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{74.000000154599547, 143}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>6</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{32.000000154599547, 143}, {32, 28}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>ID</key>
+			<integer>5</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{288.00001239776611, 12}, {111, 22}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>4</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs36 \cf0 Remote State}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+		<dict>
+			<key>Bounds</key>
+			<string>{{39.223648071289062, 12}, {91, 22}}</string>
+			<key>Class</key>
+			<string>ShapedGraphic</string>
+			<key>FitText</key>
+			<string>YES</string>
+			<key>Flow</key>
+			<string>Resize</string>
+			<key>FontInfo</key>
+			<dict>
+				<key>Font</key>
+				<string>Helvetica</string>
+				<key>Size</key>
+				<real>12</real>
+			</dict>
+			<key>ID</key>
+			<integer>3</integer>
+			<key>Shape</key>
+			<string>Rectangle</string>
+			<key>Style</key>
+			<dict>
+				<key>fill</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>shadow</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+				<key>stroke</key>
+				<dict>
+					<key>Draws</key>
+					<string>NO</string>
+				</dict>
+			</dict>
+			<key>Text</key>
+			<dict>
+				<key>Pad</key>
+				<integer>0</integer>
+				<key>Text</key>
+				<string>{\rtf1\ansi\ansicpg1252\cocoartf1187\cocoasubrtf390
+\cocoascreenfonts1{\fonttbl\f0\fswiss\fcharset0 Helvetica;}
+{\colortbl;\red255\green255\blue255;}
+\pard\tx560\tx1120\tx1680\tx2240\tx2800\tx3360\tx3920\tx4480\tx5040\tx5600\tx6160\tx6720\pardirnatural\qc
+
+\f0\fs36 \cf0 Local State}</string>
+				<key>VerticalPad</key>
+				<integer>0</integer>
+			</dict>
+			<key>Wrap</key>
+			<string>NO</string>
+		</dict>
+	</array>
+	<key>GridInfo</key>
+	<dict/>
+	<key>GuidesLocked</key>
+	<string>NO</string>
+	<key>GuidesVisible</key>
+	<string>YES</string>
+	<key>HPages</key>
+	<integer>1</integer>
+	<key>ImageCounter</key>
+	<integer>1</integer>
+	<key>KeepToScale</key>
+	<false/>
+	<key>Layers</key>
+	<array>
+		<dict>
+			<key>Lock</key>
+			<string>NO</string>
+			<key>Name</key>
+			<string>Layer 1</string>
+			<key>Print</key>
+			<string>YES</string>
+			<key>View</key>
+			<string>YES</string>
+		</dict>
+	</array>
+	<key>LayoutInfo</key>
+	<dict>
+		<key>Animate</key>
+		<string>NO</string>
+		<key>circoMinDist</key>
+		<real>18</real>
+		<key>circoSeparation</key>
+		<real>0.0</real>
+		<key>layoutEngine</key>
+		<string>dot</string>
+		<key>neatoSeparation</key>
+		<real>0.0</real>
+		<key>twopiSeparation</key>
+		<real>0.0</real>
+	</dict>
+	<key>LinksVisible</key>
+	<string>NO</string>
+	<key>MagnetsVisible</key>
+	<string>NO</string>
+	<key>MasterSheets</key>
+	<array/>
+	<key>ModificationDate</key>
+	<string>2013-07-24 21:26:31 +0000</string>
+	<key>Modifier</key>
+	<string>Jay Kreps</string>
+	<key>NotesVisible</key>
+	<string>NO</string>
+	<key>Orientation</key>
+	<integer>2</integer>
+	<key>OriginVisible</key>
+	<string>NO</string>
+	<key>PageBreaks</key>
+	<string>YES</string>
+	<key>PrintInfo</key>
+	<dict>
+		<key>NSBottomMargin</key>
+		<array>
+			<string>float</string>
+			<string>41</string>
+		</array>
+		<key>NSHorizonalPagination</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAhOU051bWJlcgCEhAdOU1ZhbHVlAISECE5TT2JqZWN0AIWEASqEhAFxlwCG</string>
+		</array>
+		<key>NSLeftMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSPaperSize</key>
+		<array>
+			<string>size</string>
+			<string>{612.00002479553223, 792}</string>
+		</array>
+		<key>NSPrintReverseOrientation</key>
+		<array>
+			<string>int</string>
+			<string>0</string>
+		</array>
+		<key>NSPrinter</key>
+		<array>
+			<string>coded</string>
+			<string>BAtzdHJlYW10eXBlZIHoA4QBQISEhAlOU1ByaW50ZXIAhIQITlNPYmplY3QAhZKEhIQITlNTdHJpbmcBlIQBKxdEZWxsIExhc2VyIFByaW50ZXIgMTcxMIaG</string>
+		</array>
+		<key>NSPrinterName</key>
+		<array>
+			<string>string</string>
+			<string>Dell Laser Printer 1710</string>
+		</array>
+		<key>NSRightMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+		<key>NSTopMargin</key>
+		<array>
+			<string>float</string>
+			<string>18</string>
+		</array>
+	</dict>
+	<key>PrintOnePage</key>
+	<false/>
+	<key>ReadOnly</key>
+	<string>NO</string>
+	<key>RowAlign</key>
+	<integer>1</integer>
+	<key>RowSpacing</key>
+	<real>36</real>
+	<key>SheetTitle</key>
+	<string>Canvas 1</string>
+	<key>SmartAlignmentGuidesActive</key>
+	<string>YES</string>
+	<key>SmartDistanceGuidesActive</key>
+	<string>YES</string>
+	<key>UniqueID</key>
+	<integer>1</integer>
+	<key>UseEntirePage</key>
+	<false/>
+	<key>VPages</key>
+	<integer>1</integer>
+	<key>WindowInfo</key>
+	<dict>
+		<key>CurrentSheet</key>
+		<integer>0</integer>
+		<key>ExpandedCanvases</key>
+		<array>
+			<dict>
+				<key>name</key>
+				<string>Canvas 1</string>
+			</dict>
+		</array>
+		<key>Frame</key>
+		<string>{{364, 6}, {711, 872}}</string>
+		<key>ListView</key>
+		<true/>
+		<key>OutlineWidth</key>
+		<integer>142</integer>
+		<key>RightSidebar</key>
+		<false/>
+		<key>ShowRuler</key>
+		<true/>
+		<key>Sidebar</key>
+		<true/>
+		<key>SidebarWidth</key>
+		<integer>120</integer>
+		<key>VisibleRegion</key>
+		<string>{{0, 0}, {576, 733}}</string>
+		<key>Zoom</key>
+		<real>1</real>
+		<key>ZoomValues</key>
+		<array>
+			<array>
+				<string>Canvas 1</string>
+				<real>1</real>
+				<real>1</real>
+			</array>
+		</array>
+	</dict>
+</dict>
+</plist>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/docs/img/0.7.0/learn/documentation/introduction/samza_state.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/introduction/samza_state.png b/docs/img/0.7.0/learn/documentation/introduction/samza_state.png
new file mode 100644
index 0000000..80a2df6
Binary files /dev/null and b/docs/img/0.7.0/learn/documentation/introduction/samza_state.png differ


[06/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT.jar
----------------------------------------------------------------------
diff --git a/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT.jar b/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT.jar
new file mode 100644
index 0000000..e2c2b2d
Binary files /dev/null and b/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT-test.jar
----------------------------------------------------------------------
diff --git a/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT-test.jar b/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT-test.jar
new file mode 100644
index 0000000..b91fc78
Binary files /dev/null and b/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT-test.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT.jar
----------------------------------------------------------------------
diff --git a/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT.jar b/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT.jar
new file mode 100644
index 0000000..d7c9bd6
Binary files /dev/null and b/samza-kafka/lib/kafka_2.9.2-0.8.1-SNAPSHOT.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
new file mode 100644
index 0000000..a9ddc5c
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint.kafka
+
+import org.I0Itec.zkclient.ZkClient
+
+import grizzled.slf4j.Logging
+import kafka.admin.AdminUtils
+import kafka.api.FetchRequestBuilder
+import kafka.api.OffsetRequest
+import kafka.api.PartitionOffsetRequestInfo
+import kafka.common.ErrorMapping
+import kafka.common.TopicAndPartition
+import kafka.common.TopicExistsException
+import kafka.consumer.SimpleConsumer
+import kafka.producer.KeyedMessage
+import kafka.producer.Partitioner
+import kafka.producer.Producer
+import kafka.serializer.Decoder
+import kafka.serializer.Encoder
+import kafka.utils.Utils
+import kafka.utils.VerifiableProperties
+import org.apache.samza.Partition
+import org.apache.samza.SamzaException
+import org.apache.samza.checkpoint.Checkpoint
+import org.apache.samza.checkpoint.CheckpointManager
+import org.apache.samza.serializers.CheckpointSerde
+import org.apache.samza.serializers.Serde
+import org.apache.samza.system.kafka.TopicMetadataCache
+import org.apache.samza.util.TopicMetadataStore
+
+/**
+ * Kafka checkpoint manager is used to store checkpoints in a Kafka topic that
+ * is uniquely identified by a job/partition combination. To read a checkpoint
+ * for a given job and partition combination (e.g. my-job, partition 1), we
+ * simply read the last message from the topic: __samza_checkpoint_my-job_1. If
+ * the topic does not yet exist, we assume that there is not yet any state for
+ * this job/partition pair, and return an empty checkpoint.
+ */
+class KafkaCheckpointManager(
+  clientId: String,
+  stateTopic: String,
+  systemName: String,
+  totalPartitions: Int,
+  replicationFactor: Int,
+  socketTimeout: Int,
+  bufferSize: Int,
+  fetchSize: Int,
+  metadataStore: TopicMetadataStore,
+  connectProducer: () => Producer[Partition, Array[Byte]],
+  connectZk: () => ZkClient,
+  failureRetryMs: Long = 10000,
+  serde: Serde[Checkpoint] = new CheckpointSerde) extends CheckpointManager with Logging {
+
+  var partitions = Set[Partition]()
+  var producer: Producer[Partition, Array[Byte]] = null
+
+  info("Creating KafkaCheckpointManager with: clientId=%s, stateTopic=%s, systemName=%s" format (clientId, stateTopic, systemName))
+
+  def writeCheckpoint(partition: Partition, checkpoint: Checkpoint) {
+    var done = false
+
+    while (!done) {
+      try {
+        if (producer == null) {
+          producer = connectProducer()
+        }
+
+        producer.send(new KeyedMessage(stateTopic, null, partition, serde.toBytes(checkpoint)))
+        done = true
+      } catch {
+        case e: Throwable =>
+          warn("Failed to send checkpoint %s for partition %s. Retrying." format (checkpoint, partition), e)
+
+          if (producer != null) {
+            producer.close
+          }
+
+          producer = null
+
+          Thread.sleep(failureRetryMs)
+      }
+    }
+  }
+
+  def readLastCheckpoint(partition: Partition): Checkpoint = {
+    var checkpoint: Option[Checkpoint] = None
+    var consumer: SimpleConsumer = null
+
+    info("Reading checkpoint for partition %s." format partition.getPartitionId)
+
+    while (!checkpoint.isDefined) {
+      try {
+        // Assume state topic exists with correct partitions, since it should be verified on start.
+        // Fetch the metadata for this state topic/partition pair.
+        val metadataMap = TopicMetadataCache.getTopicMetadata(Set(stateTopic), systemName, (topics: Set[String]) => metadataStore.getTopicInfo(topics))
+        val metadata = metadataMap(stateTopic)
+        val partitionMetadata = metadata.partitionsMetadata
+          .filter(_.partitionId == partition.getPartitionId)
+          .headOption
+          .getOrElse(throw new KafkaCheckpointException("Tried to find partition information for partition %d, but it didn't exist in Kafka." format partition.getPartitionId))
+        val partitionId = partitionMetadata.partitionId
+        val leader = partitionMetadata
+          .leader
+          .getOrElse(throw new SamzaException("No leader available for topic %s" format stateTopic))
+
+        info("Connecting to leader %s:%d for topic %s and partition %s to fetch last checkpoint message." format (leader.host, leader.port, stateTopic, partitionId))
+
+        consumer = new SimpleConsumer(
+          leader.host,
+          leader.port,
+          socketTimeout,
+          bufferSize,
+          clientId)
+        val topicAndPartition = new TopicAndPartition(stateTopic, partitionId)
+        val offset = consumer.getOffsetsBefore(new OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))))
+          .partitionErrorAndOffsets
+          .get(topicAndPartition)
+          .getOrElse(throw new KafkaCheckpointException("Unable to find offset information for %s:%d" format (stateTopic, partitionId)))
+          .offsets
+          .headOption
+          .getOrElse(throw new KafkaCheckpointException("Got response, but no offsets defined for %s:%d" format (stateTopic, partitionId)))
+
+        info("Got offset %s for topic %s and partition %s. Attempting to fetch message." format (offset, stateTopic, partitionId))
+
+        if (offset <= 0) {
+          info("Got offset 0 (no messages in state topic) for topic %s and partition %s, so returning null. If you expected the state topic to have messages, you're probably going to lose data." format (stateTopic, partition))
+          return null
+        }
+
+        val request = new FetchRequestBuilder()
+          // Kafka returns 1 greater than the offset of the last message in 
+          //the topic, so subtract one to fetch the last message.
+          .addFetch(stateTopic, partitionId, offset - 1, fetchSize)
+          .maxWait(500)
+          .minBytes(1)
+          .clientId(clientId)
+          .build
+        val messageSet = consumer.fetch(request)
+        if (messageSet.hasError) {
+          warn("Got error code from broker for %s: %s" format (stateTopic, messageSet.errorCode(stateTopic, partitionId)))
+          val errorCode = messageSet.errorCode(stateTopic, partitionId)
+          if (ErrorMapping.OffsetOutOfRangeCode.equals(errorCode)) {
+            warn("Got an offset out of range exception while getting last checkpoint for topic %s and partition %s, so returning a null offset to the KafkaConsumer. Let it decide what to do based on its autooffset.reset setting." format (stateTopic, partitionId))
+            return null
+          }
+          ErrorMapping.maybeThrowException(errorCode)
+        }
+        val messages = messageSet.messageSet(stateTopic, partitionId).toList
+
+        if (messages.length != 1) {
+          throw new KafkaCheckpointException("Something really unexpected happened. Got %s "
+            + "messages back when fetching from state checkpoint topic %s and partition %s. "
+            + "Expected one message. It would be unsafe to go on without the latest checkpoint, "
+            + "so failing." format (messages.length, stateTopic, partition))
+        }
+
+        // Some back bending to go from message to checkpoint.
+        checkpoint = Some(serde.fromBytes(Utils.readBytes(messages(0).message.payload)))
+
+        consumer.close
+      } catch {
+        case e: KafkaCheckpointException =>
+          throw e
+        case e: Throwable =>
+          warn("Got exception while trying to read last checkpoint for topic %s and partition %s. Retrying." format (stateTopic, partition), e)
+
+          if (consumer != null) {
+            consumer.close
+          }
+
+          Thread.sleep(failureRetryMs)
+      }
+    }
+
+    info("Got checkpoint state for partition %s: %s" format (partition.getPartitionId, checkpoint))
+
+    checkpoint.get
+  }
+
+  def start {
+    if (partitions.contains(new Partition(0))) {
+      createTopic
+    }
+
+    validateTopic
+  }
+
+  def register(partition: Partition) {
+    partitions += partition
+  }
+
+  def stop = producer.close
+
+  private def createTopic {
+    var done = false
+    var zkClient: ZkClient = null
+
+    info("Attempting to create state topic %s with %s partitions." format (stateTopic, totalPartitions))
+
+    while (!done) {
+      try {
+        zkClient = connectZk()
+
+        AdminUtils.createTopic(
+          zkClient,
+          stateTopic,
+          totalPartitions,
+          replicationFactor)
+
+        info("Created state topic %s." format stateTopic)
+
+        done = true
+      } catch {
+        case e: TopicExistsException =>
+          info("State topic %s already exists." format stateTopic)
+
+          done = true
+        case e: Throwable =>
+          warn("Failed to create topic %s. Retrying." format stateTopic, e)
+
+          if (zkClient != null) {
+            zkClient.close
+          }
+
+          Thread.sleep(failureRetryMs)
+      }
+    }
+
+    zkClient.close
+  }
+
+  private def validateTopic {
+    var done = false
+
+    info("Validating state topic %s." format stateTopic)
+
+    while (!done) {
+      try {
+        val topicMetadataMap = TopicMetadataCache.getTopicMetadata(Set(stateTopic), systemName, metadataStore.getTopicInfo)
+        val topicMetadata = topicMetadataMap(stateTopic)
+        val errorCode = topicMetadata.errorCode
+
+        if (errorCode != ErrorMapping.NoError) {
+          throw new SamzaException("State topic validation failed for topic %s because we got error code %s from Kafka." format (stateTopic, errorCode))
+        }
+
+        val partitionCount = topicMetadata.partitionsMetadata.length
+
+        if (partitionCount != totalPartitions) {
+          throw new KafkaCheckpointException("State topic validation failed for topic %s because partition count %s did not match expected partition count %s." format (stateTopic, topicMetadata.partitionsMetadata.length, totalPartitions))
+        }
+
+        info("Successfully validated state topic %s." format stateTopic)
+
+        done = true
+      } catch {
+        case e: KafkaCheckpointException =>
+          throw e
+        case e: Throwable =>
+          warn("Got exception while trying to read validate topic %s. Retrying." format stateTopic, e)
+
+          Thread.sleep(failureRetryMs)
+      }
+    }
+  }
+}
+
+/**
+ * KafkaCheckpointManager handles retries, so we need two kinds of exceptions:
+ * one to signal a hard failure, and the other to retry. The
+ * KafkaCheckpointException is thrown to indicate a hard failure that the Kafka
+ * CheckpointManager can't recover from.
+ */
+class KafkaCheckpointException(s: String, t: Throwable) extends SamzaException(s, t) {
+  def this(s: String) = this(s, null)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
new file mode 100644
index 0000000..bc94f6a
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint.kafka
+
+import org.apache.samza.config.{ KafkaConfig, Config }
+import org.apache.samza.SamzaException
+import java.util.Properties
+import kafka.producer.Producer
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.TaskConfig.Config2Task
+import org.apache.samza.config.KafkaConfig.Config2Kafka
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.Partition
+import grizzled.slf4j.Logging
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.util.{ KafkaUtil, ClientUtilTopicMetadataStore }
+import org.apache.samza.util.Util
+import org.I0Itec.zkclient.ZkClient
+import kafka.utils.ZKStringSerializer
+import org.apache.samza.checkpoint.CheckpointManagerFactory
+import org.apache.samza.checkpoint.CheckpointManager
+
+class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Logging {
+  def getCheckpointManager(config: Config, registry: MetricsRegistry): CheckpointManager = {
+    val clientId = KafkaUtil.getClientId("samza-checkpoint-manager", config)
+    val systemName = config
+      .getCheckpointSystem
+      .getOrElse(throw new SamzaException("no system defined for Kafka's checkpoint manager."))
+    val injectedProducerProps = Map(
+      "request.required.acks" -> "-1",
+      "producer.type" -> "sync",
+      // Subtract one here, because DefaultEventHandler calls messageSendMaxRetries + 1.
+      "message.send.max.retries" -> (Integer.MAX_VALUE - 1).toString)
+    val producerConfig = config.getKafkaSystemProducerConfig(
+      systemName,
+      clientId,
+      injectedProducerProps)
+    val consumerConfig = config.getKafkaSystemConsumerConfig(systemName, clientId)
+    val replicationFactor = config.getCheckpointReplicationFactor.getOrElse("3").toInt
+    val socketTimeout = consumerConfig.socketTimeoutMs
+    val bufferSize = consumerConfig.socketReceiveBufferBytes
+    val fetchSize = consumerConfig.fetchMessageMaxBytes // must be > buffer size
+
+    val connectProducer = () => {
+      new Producer[Partition, Array[Byte]](producerConfig)
+    }
+    val zkConnect = Option(consumerConfig.zkConnect)
+      .getOrElse(throw new SamzaException("no zookeeper.connect defined in config"))
+    val connectZk = () => {
+      new ZkClient(zkConnect, 6000, 6000, ZKStringSerializer)
+    }
+    val jobName = config.getName.getOrElse(throw new SamzaException("Missing job name in configs"))
+    val jobId = config.getJobId.getOrElse("1")
+    val brokersListString = Option(producerConfig.brokerList)
+      .getOrElse(throw new SamzaException("No broker list defined in config for %s." format systemName))
+    val metadataStore = new ClientUtilTopicMetadataStore(brokersListString, clientId)
+    val stateTopic = getTopic(jobName, jobId)
+    val totalPartitions = Util.getMaxInputStreamPartitions(config).size
+
+    new KafkaCheckpointManager(
+      clientId,
+      stateTopic,
+      systemName,
+      totalPartitions,
+      replicationFactor,
+      socketTimeout,
+      bufferSize,
+      fetchSize,
+      metadataStore,
+      connectProducer,
+      connectZk)
+  }
+
+  private def getTopic(jobName: String, jobId: String) =
+    "__samza_checkpoint_%s_%s" format (jobName.replaceAll("_", "-"), jobId.replaceAll("_", "-"))
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
new file mode 100644
index 0000000..59d915d
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import scala.collection.JavaConversions._
+import kafka.consumer.ConsumerConfig
+import java.util.Properties
+import kafka.producer.ProducerConfig
+import java.util.UUID
+
+object KafkaConfig {
+  val REGEX_RESOLVED_STREAMS = "job.config.rewriter.%s.regex"
+  val REGEX_RESOLVED_SYSTEM = "job.config.rewriter.%s.system"
+  val REGEX_INHERITED_CONFIG = "job.config.rewriter.%s.config"
+
+  val CHECKPOINT_SYSTEM = "task.checkpoint.system"
+  val CHECKPOINT_REPLICATION_FACTOR = "task.checkpoint.replication.factor"
+
+  val CONSUMER_KEY_DESERIALIZER = SystemConfig.SYSTEM_PREFIX + "consumer.key.deserializer.class"
+  val CONSUMER_MSG_DESERIALIZER = SystemConfig.SYSTEM_PREFIX + "consumer.deserializer.class"
+
+  implicit def Config2Kafka(config: Config) = new KafkaConfig(config)
+}
+
+class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
+  // checkpoints
+  def getCheckpointSystem = getOption(KafkaConfig.CHECKPOINT_SYSTEM)
+  def getCheckpointReplicationFactor() = getOption(KafkaConfig.CHECKPOINT_REPLICATION_FACTOR)
+
+  // custom consumer config
+  def getConsumerKeyDeserializerClass(name: String) = getOption(KafkaConfig.CONSUMER_KEY_DESERIALIZER format name)
+  def getConsumerMsgDeserializerClass(name: String) = getOption(KafkaConfig.CONSUMER_MSG_DESERIALIZER format name)
+
+  /**
+   * Returns a map of topic -> auto.offset.reset value for all streams that
+   * are defined with this property in the config.
+   */
+  def getAutoOffsetResetTopics(systemName: String) = {
+    val subConf = config.subset("systems.%s.streams." format systemName, true)
+    // find all .samza.partition.manager keys, and strip the suffix
+    subConf
+      .filterKeys(k => k.endsWith(".consumer.auto.offset.reset"))
+      .map {
+        case (topicAutoOffsetReset, resetValue) =>
+          (topicAutoOffsetReset.replace(".consumer.auto.offset.reset", ""), resetValue)
+      }.toMap
+  }
+
+  // regex resolver
+  def getRegexResolvedStreams(rewriterName: String) = getOption(KafkaConfig.REGEX_RESOLVED_STREAMS format rewriterName)
+  def getRegexResolvedSystem(rewriterName: String) = getOption(KafkaConfig.REGEX_RESOLVED_SYSTEM format rewriterName)
+  def getRegexResolvedInheritedConfig(rewriterName: String) = config.subset((KafkaConfig.REGEX_INHERITED_CONFIG format rewriterName) + ".", true)
+
+  // kafka config
+  def getKafkaSystemConsumerConfig(
+    systemName: String,
+    clientId: String = "undefined-samza-consumer-" format UUID.randomUUID.toString,
+    groupId: String = "undefined-samza-consumer-group-" format UUID.randomUUID.toString,
+    injectedProps: Map[String, String] = Map()) = {
+
+    val subConf = config.subset("systems.%s.consumer." format systemName, true)
+    val consumerProps = new Properties()
+    consumerProps.putAll(subConf)
+    consumerProps.put("group.id", groupId)
+    consumerProps.put("client.id", clientId)
+    consumerProps.putAll(injectedProps)
+    new ConsumerConfig(consumerProps)
+  }
+ 
+  def getKafkaSystemProducerConfig(
+    systemName: String,
+    clientId: String = "undefined-samza-producer-" format UUID.randomUUID.toString,
+    injectedProps: Map[String, String] = Map()) = {
+
+    val subConf = config.subset("systems.%s.producer." format systemName, true)
+    val producerProps = new Properties()
+    producerProps.putAll(subConf)
+    producerProps.put("client.id", clientId)
+    producerProps.putAll(injectedProps)
+    new ProducerConfig(producerProps)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/config/KafkaSerdeConfig.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaSerdeConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaSerdeConfig.scala
new file mode 100644
index 0000000..11078e3
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaSerdeConfig.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+object KafkaSerdeConfig {
+  // kafka serde config constants
+  val ENCODER = SerializerConfig.SERIALIZER_PREFIX + ".encoder"
+  val DECODER = SerializerConfig.SERIALIZER_PREFIX + ".decoder"
+
+  implicit def Config2KafkaSerde(config: Config) = new KafkaSerdeConfig(config)
+}
+
+class KafkaSerdeConfig(config: Config) extends ScalaMapConfig(config) {
+  def getKafkaEncoder(serializer: String) =
+    getOption(KafkaSerdeConfig.ENCODER format serializer)
+
+  def getKafkaDecoder(serializer: String) =
+    getOption(KafkaSerdeConfig.DECODER format serializer)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala b/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala
new file mode 100644
index 0000000..c7c50c4
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/RegExTopicGenerator.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import org.I0Itec.zkclient.ZkClient
+import kafka.utils.{ ZkUtils, ZKStringSerializer }
+import org.apache.samza.config.KafkaConfig.{ Config2Kafka, REGEX_RESOLVED_STREAMS }
+import org.apache.samza.SamzaException
+import collection.JavaConversions._
+import grizzled.slf4j.Logging
+import scala.collection._
+import org.apache.samza.config.TaskConfig.Config2Task
+import org.apache.samza.system.SystemStream
+import org.apache.samza.util.Util
+
+/**
+ * Dynamically determine the Kafka topics to use as input streams to the task via a regular expression.
+ * For each topic that matches the regular expression, generate a series of config values for it and
+ * add it to the task's input streams setting.
+ *
+ * job.config.rewriter.regex-input-rewriter.regex=.*stream
+ * job.config.rewriter.regex-input-rewriter.system=kafka
+ * job.config.rewriter.regex-input-rewriter.config.foo=bar
+ *
+ * Would result in:
+ *
+ * task.inputs=kafka.somestream
+ * systems.kafka.streams.somestream.foo=bar
+ *
+ * @see samza.config.KafkaConfig.getRegexResolvedStreams
+ *
+ */
+class RegexTopicGenerator extends ConfigRewriter with Logging {
+
+  def rewrite(rewriterName: String, config: Config): Config = {
+    val regex = config
+      .getRegexResolvedStreams(rewriterName)
+      .getOrElse(throw new SamzaException("No %s defined in config" format REGEX_RESOLVED_STREAMS))
+    val systemName = config
+      .getRegexResolvedSystem(rewriterName)
+      .getOrElse(throw new SamzaException("No system defined for %s." format rewriterName))
+    val topics = getTopicsFromZK(rewriterName, config)
+    val existingInputStreams = config.getInputStreams
+    val newInputStreams = new mutable.HashSet[SystemStream]
+    val keysAndValsToAdd = new mutable.HashMap[String, String]
+
+    // Find all the topics that match this regex
+    val matchingStreams = topics
+      .filter(_.matches(regex))
+      .map(new SystemStream(systemName, _))
+      .toSet
+
+    for (m <- matchingStreams) {
+      info("Generating new configs for matching stream %s." format m)
+
+      if (existingInputStreams.contains(m)) {
+        throw new SamzaException("Regex '%s' matches existing, statically defined input %s." format (regex, m))
+      }
+
+      newInputStreams.add(m)
+
+      // For each topic that matched, generate all the specified configs
+      config
+        .getRegexResolvedInheritedConfig(rewriterName)
+        .foreach(kv => keysAndValsToAdd.put("systems." + m.getSystem + ".streams." + m.getStream + "." + kv._1, kv._2))
+    }
+    // Build new inputs
+
+    info("Generated config values for %d new topics" format newInputStreams.size)
+
+    val inputStreams = TaskConfig.INPUT_STREAMS -> (existingInputStreams ++ newInputStreams)
+      .map(Util.getNameFromSystemStream(_))
+      .mkString(",")
+
+    new MapConfig((keysAndValsToAdd ++ config) += inputStreams)
+  }
+
+  def getTopicsFromZK(rewriterName: String, config: Config): Seq[String] = {
+    val systemName = config
+      .getRegexResolvedSystem(rewriterName)
+      .getOrElse(throw new SamzaException("No system defined in config for rewriter %s." format rewriterName))
+    val consumerConfig = config.getKafkaSystemConsumerConfig(systemName)
+    val zkConnect = Option(consumerConfig.zkConnect)
+      .getOrElse(throw new SamzaException("No zookeeper.connect for system %s defined in config." format systemName))
+    val zkClient = new ZkClient(zkConnect, 6000, 6000, ZKStringSerializer)
+
+    try {
+      ZkUtils.getAllTopics(zkClient)
+    } finally {
+      zkClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/serializers/KafkaSerde.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/serializers/KafkaSerde.scala b/samza-kafka/src/main/scala/org/apache/samza/serializers/KafkaSerde.scala
new file mode 100644
index 0000000..82ba2a0
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/serializers/KafkaSerde.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+import java.nio.ByteBuffer
+import org.apache.samza.util.Util
+import kafka.serializer.Encoder
+import kafka.serializer.Decoder
+import org.apache.samza.config.Config
+import org.apache.samza.config.KafkaSerdeConfig.Config2KafkaSerde
+import org.apache.samza.SamzaException
+
+class KafkaSerde[T](encoder: Encoder[T], decoder: Decoder[T]) extends Serde[T] {
+  def toBytes(obj: T): Array[Byte] = encoder.toBytes(obj)
+  def fromBytes(bytes: Array[Byte]): T = decoder.fromBytes(bytes)
+}
+
+class KafkaSerdeFactory[T] extends SerdeFactory[T] {
+  def getSerde(name: String, config: Config): Serde[T] = {
+    val encoderClassName = config
+      .getKafkaEncoder(name)
+      .getOrElse(throw new SamzaException("No kafka encoder defined for %s" format name))
+    val decoderClassName = config
+      .getKafkaDecoder(name)
+      .getOrElse(throw new SamzaException("No kafka decoder defined for %s" format name))
+    val encoder = Util.getObj[Encoder[T]](encoderClassName)
+    val decoder = Util.getObj[Decoder[T]](decoderClassName)
+    new KafkaSerde(encoder, decoder)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala
new file mode 100644
index 0000000..cb5015d
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxy.scala
@@ -0,0 +1,215 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import kafka.consumer.SimpleConsumer
+import kafka.api._
+import kafka.common.ErrorMapping
+import java.util.concurrent.{ CountDownLatch, ConcurrentHashMap }
+import scala.collection.JavaConversions._
+import org.apache.samza.config.Config
+import org.apache.samza.util.KafkaUtil
+import org.apache.samza.config.KafkaConfig.Config2Kafka
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.metrics.MetricsRegistry
+import kafka.common.TopicAndPartition
+import kafka.message.MessageSet
+import grizzled.slf4j.Logging
+import java.nio.channels.ClosedByInterruptException
+
+/**
+ * A BrokerProxy consolidates Kafka fetches meant for a particular broker and retrieves them all at once, providing
+ * a way for consumers to retrieve those messages by topic and partition.
+ */
+abstract class BrokerProxy(
+  val host: String,
+  val port: Int,
+  val system: String,
+  val clientID: String,
+  val metricsRegistry: MetricsRegistry,
+  tpMetrics: TopicAndPartitionMetrics,
+  val timeout: Int = Int.MaxValue,
+  val bufferSize: Int = 1024000,
+  offsetGetter: GetOffset = new GetOffset("fail")) extends Toss with Logging with BrokerProxyMetrics {
+
+  val messageSink: MessageSink
+
+  /**
+   * How long should the fetcher thread sleep before checking if any TopicPartitions has been added to its purview
+   */
+  val sleepMSWhileNoTopicPartitions = 1000
+
+  /** What's the next offset for a particular partition? **/
+  val nextOffsets: ConcurrentHashMap[TopicAndPartition, Long] = new ConcurrentHashMap[TopicAndPartition, Long]()
+
+  /** Block on the first call to get message if the fetcher has not yet returned its initial results **/
+  // TODO: It should be sufficient to just use the count down latch and await on it for each of the calls, but
+  // VisualVM was showing the consumer thread spending all its time in the await method rather than returning
+  // immediately, even though the process was proceeding normally.  Hence the extra boolean.  Should be investigated.
+  val firstCallBarrier = new CountDownLatch(1)
+  var firstCall = true
+
+  var simpleConsumer = createSimpleConsumer()
+
+  def createSimpleConsumer() = {
+    val hostString = "%s:%d" format (host, port)
+    info("Creating new SimpleConsumer for host %s for system %s" format (hostString, system))
+
+    val sc = new SimpleConsumer(host, port, timeout, bufferSize, clientID) with DefaultFetch {
+      val fetchSize: Int = 256 * 1024
+    }
+
+    sc
+  }
+
+  def addTopicPartition(tp: TopicAndPartition, lastCheckpointedOffset: String) = {
+    debug("Adding new topic and partition %s to queue for %s" format (tp, host))
+    if (nextOffsets.containsKey(tp)) toss("Already consuming TopicPartition %s" format tp)
+
+    val offset = offsetGetter.getNextOffset(simpleConsumer, tp, lastCheckpointedOffset)
+    nextOffsets += tp -> offset
+
+    tpGaugeInc
+  }
+
+  def removeTopicPartition(tp: TopicAndPartition) = {
+    if (nextOffsets.containsKey(tp)) {
+      nextOffsets.remove(tp)
+      tpGaugeDec
+      debug("Removed %s" format tp)
+    } else {
+      warn("Asked to remove topic and partition %s, but not in map (keys = %s)" format (tp, nextOffsets.keys().mkString(",")))
+    }
+  }
+
+  val thread: Thread = new Thread(new Runnable() {
+    def run() {
+      info("Starting thread for BrokerProxy")
+
+      while (!Thread.currentThread.isInterrupted) {
+        if (nextOffsets.size == 0) {
+          debug("No TopicPartitions to fetch. Sleeping.")
+          Thread.sleep(sleepMSWhileNoTopicPartitions)
+        } else {
+          try {
+            fetchMessages()
+          } catch {
+            // If we're interrupted, don't try and reconnect. We should shut down.
+            case e: InterruptedException =>
+              debug("Shutting down due to interrupt exception.")
+              Thread.currentThread.interrupt
+            case e: ClosedByInterruptException =>
+              debug("Shutting down due to closed by interrupt exception.")
+              Thread.currentThread.interrupt
+            case e: Throwable => {
+              warn("Recreating simple consumer and retrying connection")
+              debug("Stack trace for fetchMessages exception.", e)
+              simpleConsumer.close()
+              simpleConsumer = createSimpleConsumer()
+              reconnectCounter.inc
+            }
+          }
+        }
+      }
+
+    }
+  }, "BrokerProxy thread pointed at %s:%d for client %s" format (host, port, clientID))
+
+  private def fetchMessages(): Unit = {
+    val response: FetchResponse = simpleConsumer.defaultFetch(nextOffsets.toList: _*)
+    firstCall = false
+    firstCallBarrier.countDown()
+    if (response.hasError) {
+      // FetchResponse should really return Option and a list of the errors so we don't have to find them ourselves
+      case class Error(tp: TopicAndPartition, code: Short, exception: Throwable)
+
+      val errors = for (
+        error <- response.data.entrySet.filter(_.getValue.error != ErrorMapping.NoError);
+        errorCode <- Option(response.errorCode(error.getKey.topic, error.getKey.partition)); // Scala's being cranky about referring to error.getKey values...
+        exception <- Option(ErrorMapping.exceptionFor(errorCode))
+      ) yield new Error(error.getKey, errorCode, exception)
+
+      val (notLeaders, otherErrors) = errors.partition(_.code == ErrorMapping.NotLeaderForPartitionCode)
+
+      if (!notLeaders.isEmpty) {
+        info("Abdicating. Got not leader exception for: " + notLeaders.mkString(","))
+
+        notLeaders.foreach(e => {
+          // Go back one message, since the fetch for nextOffset failed, and 
+          // abdicate requires lastOffset, not nextOffset.
+          messageSink.abdicate(e.tp, nextOffsets.remove(e.tp) - 1)
+        })
+      }
+
+      if (!otherErrors.isEmpty) {
+        warn("Got error codes during multifetch. Throwing an exception to trigger reconnect. Errors: %s" format errors.mkString(","))
+        otherErrors.foreach(e => ErrorMapping.maybeThrowException(e.code)) // One will get thrown
+      }
+    }
+
+    def moveMessagesToTheirQueue(tp: TopicAndPartition, data: FetchResponsePartitionData) = {
+      val messageSet: MessageSet = data.messages
+      var nextOffset = nextOffsets(tp)
+
+      messageSink.setIsAtHighWatermark(tp, data.hw == 0 || data.hw == nextOffset)
+
+      for (message <- messageSet.iterator) {
+        messageSink.addMessage(tp, message, data.hw) // TODO: Verify this is correct
+
+        nextOffset = message.nextOffset
+
+        tpMetrics.getReadsCounter(tp).inc
+        tpMetrics.getBytesReadCounter(tp).inc(message.message.payloadSize + message.message.keySize)
+        tpMetrics.getOffsetCounter(tp).set(nextOffset)
+      }
+
+      nextOffsets.replace(tp, nextOffset) // use replace rather than put in case this tp was removed while we were fetching.
+
+      // Update high water mark
+      val hw = data.hw
+      if (hw >= 0) {
+        getLagGauge(tp).set(hw - nextOffset)
+      } else {
+        debug("Got a high water mark less than 0 (%d) for %s, so skipping." format (hw, tp))
+      }
+    }
+
+    response.data.foreach { case (tp, data) => moveMessagesToTheirQueue(tp, data) }
+
+  }
+
+  override def toString() = "BrokerProxy for %s:%d" format (host, port)
+
+  def start {
+    debug("Starting broker proxy for %s:%s." format (host, port))
+
+    thread.setDaemon(true)
+    thread.start
+  }
+
+  def stop {
+    debug("Shutting down broker proxy for %s:%s." format (host, port))
+
+    thread.interrupt
+    thread.join
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxyMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxyMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxyMetrics.scala
new file mode 100644
index 0000000..bdd91da
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/BrokerProxyMetrics.scala
@@ -0,0 +1,52 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.metrics.{Gauge, Counter, MetricsRegistry}
+import collection.mutable
+import kafka.common.TopicAndPartition
+
+private[kafka] trait BrokerProxyMetrics {
+  self:BrokerProxy =>
+  // TODO: Move topic-partition specific metrics out of brokerproxy, into system
+  val metricsRegistry:MetricsRegistry
+
+  def newCounter = metricsRegistry.newCounter(metricsGroup, _:String)
+
+  val hostPort = host + ":"  + port
+  val metricsGroup = "samza.kafka.brokerproxy"
+
+  // Counters
+  val reconnectCounter = newCounter("%s-Reconnects" format hostPort)
+
+
+
+  // Gauges
+  val lagGauges = mutable.Map[TopicAndPartition, Gauge[Long]]()
+  def getLagGauge(tp:TopicAndPartition) = lagGauges.getOrElseUpdate(tp, metricsRegistry.newGauge[Long](metricsGroup, "%s-MessagesBehindHighWaterMark" format tp, 0l))
+
+  val tpGauge = metricsRegistry.newGauge[Long](metricsGroup, "%s-NumberOfTopicsPartitions" format hostPort, 0)
+
+  def tpGaugeInc = tpGauge.set(tpGauge.getValue + 1l)
+
+  def tpGaugeDec = tpGauge.set(tpGauge.getValue - 1l)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/DefaultFetch.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/DefaultFetch.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/DefaultFetch.scala
new file mode 100644
index 0000000..41710f2
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/DefaultFetch.scala
@@ -0,0 +1,48 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import kafka.consumer.SimpleConsumer
+import kafka.api.FetchRequestBuilder
+import kafka.common.TopicAndPartition
+
+/**
+ * Extension to a SimpleConsumer that defines the default parameters necessary for default fetch requests.  Builds
+ * such a fetch request, requests the fetch and returns the result
+ */
+trait DefaultFetch {
+  self:SimpleConsumer =>
+  val maxWait:Int = Int.MaxValue
+  val minBytes:Int = 1
+  val clientId:String
+  val fetchSize:Int
+
+  def defaultFetch(fetches:(TopicAndPartition, Long)*) = {
+    val fbr = new FetchRequestBuilder().maxWait(1000)
+                                       .minBytes(minBytes)
+                                       .clientId(clientId)
+
+    fetches.foreach(f => fbr.addFetch(f._1.topic, f._1.partition, f._2, fetchSize))
+
+    this.fetch(fbr.build())
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala
new file mode 100644
index 0000000..326d6c9
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/GetOffset.scala
@@ -0,0 +1,94 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import kafka.consumer.SimpleConsumer
+import kafka.common.{ OffsetOutOfRangeException, ErrorMapping }
+import kafka.api._
+import org.apache.samza.config.KafkaConfig
+import org.apache.samza.config.KafkaConfig.Config2Kafka
+import kafka.common.TopicAndPartition
+import kafka.api.PartitionOffsetRequestInfo
+import grizzled.slf4j.Logging
+
+class GetOffset(default: String, autoOffsetResetTopics: Map[String, String] = Map()) extends Logging with Toss {
+
+  private def getAutoOffset(topic: String): Long = {
+    info("Checking if auto.offset.reset is defined for topic %s" format (topic))
+    autoOffsetResetTopics.getOrElse(topic, default) match {
+      case OffsetRequest.LargestTimeString =>
+        info("Got reset of type %s." format OffsetRequest.LargestTimeString)
+        OffsetRequest.LatestTime
+      case OffsetRequest.SmallestTimeString =>
+        info("Got reset of type %s." format OffsetRequest.SmallestTimeString)
+        OffsetRequest.EarliestTime
+      case other => toss("Can't get offset value for topic %s due to invalid value: %s" format (topic, other))
+    }
+  }
+
+  /**
+   *  An offset was provided but may not be valid.  Verify its validity.
+   */
+  private def useLastCheckpointedOffset(sc: DefaultFetch, last: String, tp: TopicAndPartition): Option[Long] = {
+    try {
+      info("Validating offset %s for topic and partition %s" format (last, tp))
+
+      val messages = sc.defaultFetch((tp, last.toLong))
+
+      if (messages.hasError) {
+        ErrorMapping.maybeThrowException(messages.errorCode(tp.topic, tp.partition))
+      }
+
+      info("Able to successfully read from offset %s for topic and partition %s. Using it to instantiate consumer." format (last, tp))
+
+      val nextOffset = messages
+        .messageSet(tp.topic, tp.partition)
+        .head
+        .nextOffset
+
+      info("Got next offset %s for %s." format (nextOffset, tp))
+
+      Some(nextOffset)
+    } catch {
+      case e: OffsetOutOfRangeException =>
+        info("An out of range Kafka offset (%s) was supplied for topic and partition %s, so falling back to autooffset.reset." format (last, tp))
+        None
+    }
+  }
+
+  def getNextOffset(sc: SimpleConsumer with DefaultFetch, tp: TopicAndPartition, lastCheckpointedOffset: String): Long = {
+    val offsetRequest = new OffsetRequest(Map(tp -> new PartitionOffsetRequestInfo(getAutoOffset(tp.topic), 1)))
+    val offsetResponse = sc.getOffsetsBefore(offsetRequest)
+    val partitionOffsetResponse = offsetResponse.partitionErrorAndOffsets.get(tp).getOrElse(toss("Unable to find offset information for %s" format tp))
+    val autoOffset = partitionOffsetResponse.offsets.headOption.getOrElse(toss("Got response, but no offsets defined for %s" format tp))
+
+    info("Got offset %d for topic and partition %s" format (autoOffset, tp))
+
+    val actualOffset = Option(lastCheckpointedOffset) match {
+      case Some(last) => useLastCheckpointedOffset(sc, last, tp).getOrElse(autoOffset)
+      case None => autoOffset
+    }
+
+    info("Final offset to be returned for Topic and Partition %s = %d" format (tp, actualOffset))
+    actualOffset
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
new file mode 100644
index 0000000..183c6cc
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.Partition
+import java.util.UUID
+import org.apache.samza.util.ClientUtilTopicMetadataStore
+import kafka.api.TopicMetadata
+import scala.collection.JavaConversions._
+import org.apache.samza.system.SystemAdmin
+
+class KafkaSystemAdmin(
+  systemName: String,
+  // TODO whenever Kafka decides to make the Set[Broker] class public, let's switch to Set[Broker] here.
+  brokerListString: String,
+  clientId: String = UUID.randomUUID.toString) extends SystemAdmin {
+
+  def getPartitions(streamName: String): java.util.Set[Partition] = {
+    val getTopicMetadata = (topics: Set[String]) => {
+      new ClientUtilTopicMetadataStore(brokerListString, clientId)
+        .getTopicInfo(topics)
+    }
+
+    val metadata = TopicMetadataCache.getTopicMetadata(
+      Set(streamName),
+      systemName,
+      getTopicMetadata)
+
+    metadata(streamName)
+      .partitionsMetadata
+      .map(pm => new Partition(pm.partitionId))
+      .toSet[Partition]
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.scala
new file mode 100644
index 0000000..bd7794a
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.scala
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.util.{ KafkaUtil, ClientUtilTopicMetadataStore }
+import kafka.common.TopicAndPartition
+import org.apache.samza.config.{ KafkaConfig, Config }
+import org.apache.samza.SamzaException
+import org.apache.samza.config.KafkaConfig.Config2Kafka
+import org.apache.samza.metrics.MetricsRegistry
+import grizzled.slf4j.Logging
+import scala.collection.JavaConversions._
+import kafka.message.MessageAndOffset
+import org.apache.samza.Partition
+import kafka.utils.Utils
+import org.apache.samza.util.Clock
+import java.util.UUID
+import kafka.serializer.DefaultDecoder
+import kafka.serializer.Decoder
+import org.apache.samza.util.BlockingEnvelopeMap
+import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.system.IncomingMessageEnvelope
+import java.nio.charset.Charset
+
+object KafkaSystemConsumer {
+  def toTopicAndPartition(systemStreamPartition: SystemStreamPartition) = {
+    val topic = systemStreamPartition.getStream
+    val partitionId = systemStreamPartition.getPartition.getPartitionId
+    TopicAndPartition(topic, partitionId)
+  }
+}
+
+/**
+ *  Maintain a cache of BrokerProxies, returning the appropriate one for the
+ *  requested topic and partition.
+ */
+private[kafka] class KafkaSystemConsumer(
+  systemName: String,
+  brokerListString: String,
+  metricsRegistry: MetricsRegistry,
+  clientId: String = "undefined-client-id-" + UUID.randomUUID.toString,
+  queueSize: Int = 1000,
+  timeout: Int = Int.MaxValue,
+  bufferSize: Int = 1024000,
+  brokerMetadataFailureRefreshMs: Long = 10000,
+  offsetGetter: GetOffset = new GetOffset("fail"),
+  deserializer: Decoder[Object] = new DefaultDecoder().asInstanceOf[Decoder[Object]],
+  keyDeserializer: Decoder[Object] = new DefaultDecoder().asInstanceOf[Decoder[Object]],
+  clock: () => Long = { System.currentTimeMillis }) extends BlockingEnvelopeMap(queueSize, metricsRegistry, new Clock {
+  def currentTimeMillis = clock()
+}) with Toss with Logging {
+
+  type HostPort = (String, Int)
+  val brokerProxies = scala.collection.mutable.Map[HostPort, BrokerProxy]()
+  var lastReadOffsets = Map[SystemStreamPartition, String]()
+  val topicAndPartitionMetrics = new TopicAndPartitionMetrics(metricsRegistry)
+
+  def start() {
+    val topicPartitionsAndOffsets = lastReadOffsets.map {
+      case (systemStreamPartition, offset) =>
+        val topicAndPartition = KafkaSystemConsumer.toTopicAndPartition(systemStreamPartition)
+        topicAndPartitionMetrics.addNewTopicAndPartition(topicAndPartition)
+        (topicAndPartition, offset)
+    }
+
+    refreshBrokers(topicPartitionsAndOffsets)
+
+    brokerProxies.values.foreach(_.start)
+  }
+
+  override def register(systemStreamPartition: SystemStreamPartition, lastReadOffset: String) {
+    super.register(systemStreamPartition, lastReadOffset)
+
+    lastReadOffsets += systemStreamPartition -> lastReadOffset
+  }
+
+  def stop() {
+    brokerProxies.values.foreach(_.stop)
+  }
+
+  def refreshBrokers(topicPartitionsAndOffsets: Map[TopicAndPartition, String]) {
+    var done = false
+
+    while (!done) {
+      try {
+        val getTopicMetadata = (topics: Set[String]) => {
+          new ClientUtilTopicMetadataStore(brokerListString, clientId).getTopicInfo(topics)
+        }
+
+        val partitionMetadata = TopicMetadataCache.getTopicMetadata(
+          topicPartitionsAndOffsets.keys.map(_.topic).toSet,
+          systemName,
+          getTopicMetadata)
+
+        topicPartitionsAndOffsets.map {
+          case (topicAndPartition, lastOffset) =>
+            // TODO whatever we do, we can't say Broker, even though we're 
+            // manipulating it here. Broker is a private type and Scala doesn't seem 
+            // to care about that as long as you don't explicitly declare its type.
+            val brokerOption = partitionMetadata(topicAndPartition.topic)
+              .partitionsMetadata
+              .find(_.partitionId == topicAndPartition.partition)
+              .getOrElse(toss("Can't find leader for %s" format topicAndPartition))
+              .leader
+
+            brokerOption match {
+              case Some(broker) =>
+                val brokerProxy = brokerProxies.getOrElseUpdate((broker.host, broker.port), new BrokerProxy(broker.host, broker.port, systemName, clientId, metricsRegistry, topicAndPartitionMetrics, timeout, bufferSize, offsetGetter) {
+                  val messageSink: MessageSink = sink
+                })
+
+                brokerProxy.addTopicPartition(topicAndPartition, lastOffset)
+              case _ => warn("Broker for %s not defined! " format topicAndPartition)
+            }
+        }
+
+        done = true
+      } catch {
+        case e: Throwable =>
+          warn("An exception was thrown while refreshing brokers for %s. Waiting a bit and retrying, since we can't continue without broker metadata." format topicPartitionsAndOffsets.keySet)
+          debug(e)
+
+          try {
+            Thread.sleep(brokerMetadataFailureRefreshMs)
+          } catch {
+            case e: InterruptedException =>
+              info("Interrupted while waiting to retry metadata refresh, so shutting down.")
+
+              stop
+          }
+      }
+    }
+  }
+
+  val sink = new MessageSink {
+    def setIsAtHighWatermark(tp: TopicAndPartition, isAtHighWatermark: Boolean) {
+      setIsAtHead(toSystemStreamPartition(tp), isAtHighWatermark)
+    }
+
+    def addMessage(tp: TopicAndPartition, msg: MessageAndOffset, highWatermark: Long) = {
+      trace("Incoming message %s: %s." format (tp, msg))
+
+      val systemStreamPartition = toSystemStreamPartition(tp)
+      val isAtHead = highWatermark == msg.offset
+      val offset = msg.offset.toString
+      val key = if (msg.message.key != null) {
+        keyDeserializer.fromBytes(Utils.readBytes(msg.message.key))
+      } else {
+        null
+      }
+      val message = if (msg.message.buffer != null) {
+        deserializer.fromBytes(Utils.readBytes(msg.message.payload))
+      } else {
+        null
+      }
+
+      // TODO use kafka encoder/decoder here, if they were defined in config
+      add(systemStreamPartition, new IncomingMessageEnvelope(systemStreamPartition, offset, key, message))
+
+      setIsAtHead(systemStreamPartition, isAtHead)
+    }
+
+    def abdicate(tp: TopicAndPartition, lastOffset: Long) {
+      refreshBrokers(Map(tp -> lastOffset.toString))
+    }
+
+    private def toSystemStreamPartition(tp: TopicAndPartition) = {
+      new SystemStreamPartition(systemName, tp.topic, new Partition(tp.partition))
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemFactory.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemFactory.scala
new file mode 100644
index 0000000..13c5baa
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemFactory.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.util.KafkaUtil
+import org.apache.samza.config.Config
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.config.KafkaConfig
+import org.apache.samza.config.KafkaConfig.Config2Kafka
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.util.ClientUtilTopicMetadataStore
+import org.apache.samza.SamzaException
+import scala.collection.JavaConversions._
+import java.util.Properties
+import kafka.producer.Producer
+import kafka.producer.async.DefaultEventHandler
+import kafka.utils.Utils
+import org.apache.samza.util.Util
+import kafka.serializer.Decoder
+import kafka.serializer.DefaultDecoder
+import org.apache.samza.system.SystemFactory
+
+class KafkaSystemFactory extends SystemFactory {
+  def getConsumer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    val clientId = KafkaUtil.getClientId("samza-consumer", config)
+
+    // Kind of goofy to need a producer config for consumers, but we need metadata.
+    val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId)
+    val brokerListString = Option(producerConfig.brokerList)
+      .getOrElse(throw new SamzaException("No broker list defined in config for %s." format systemName))
+    val consumerConfig = config.getKafkaSystemConsumerConfig(systemName, clientId)
+
+    // TODO could add stream-level overrides for timeout and buffer size
+    val timeout = consumerConfig.socketTimeoutMs
+    val bufferSize = consumerConfig.socketReceiveBufferBytes
+    val autoOffsetResetDefault = consumerConfig.autoOffsetReset
+    val autoOffsetResetTopics = config.getAutoOffsetResetTopics(systemName)
+    val offsetGetter = new GetOffset(autoOffsetResetDefault, autoOffsetResetTopics)
+    val deserializer = config.getConsumerMsgDeserializerClass(systemName) match {
+      case Some(deserializerClass) => Util.getObj[Decoder[Object]](deserializerClass)
+      case _ => new DefaultDecoder().asInstanceOf[Decoder[Object]]
+    }
+    val keyDeserializer = config.getConsumerKeyDeserializerClass(systemName) match {
+      case Some(deserializerClass) => Util.getObj[Decoder[Object]](deserializerClass)
+      case _ => new DefaultDecoder().asInstanceOf[Decoder[Object]]
+    }
+
+    new KafkaSystemConsumer(
+      systemName = systemName,
+      brokerListString = brokerListString,
+      metricsRegistry = registry,
+      clientId = clientId,
+      // TODO make this configurable?
+      queueSize = 1000,
+      timeout = timeout,
+      bufferSize = bufferSize,
+      offsetGetter = offsetGetter)
+  }
+
+  def getProducer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    val clientId = KafkaUtil.getClientId("samza-producer", config)
+    val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId)
+    val batchSize = Option(producerConfig.batchNumMessages)
+      .getOrElse(1000)
+    val reconnectIntervalMs = Option(producerConfig.retryBackoffMs)
+      .getOrElse(10000)
+    val getProducer = () => { new Producer[Object, Object](producerConfig) }
+
+    // Unlike consumer, no need to use encoders here, since they come for free 
+    // inside the producer configs. Kafka's producer will handle all of this 
+    // for us.
+
+    new KafkaSystemProducer(
+      systemName,
+      batchSize,
+      reconnectIntervalMs,
+      registry,
+      getProducer)
+  }
+
+  def getAdmin(systemName: String, config: Config) = {
+    val clientId = KafkaUtil.getClientId("samza-admin", config)
+    val producerConfig = config.getKafkaSystemProducerConfig(systemName, clientId)
+    val brokerListString = Option(producerConfig.brokerList)
+      .getOrElse(throw new SamzaException("No broker list defined in config for %s." format systemName))
+
+    new KafkaSystemAdmin(systemName, brokerListString, clientId)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala
new file mode 100644
index 0000000..c4e4bec
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemProducer.scala
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import java.nio.ByteBuffer
+import java.util.Properties
+import scala.collection.mutable.ArrayBuffer
+import grizzled.slf4j.Logging
+import kafka.producer.KeyedMessage
+import kafka.producer.Producer
+import kafka.producer.ProducerConfig
+import org.apache.samza.config.Config
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.util.KafkaUtil
+import org.apache.samza.system.SystemProducer
+import org.apache.samza.system.OutgoingMessageEnvelope
+
+object KafkaSystemProducerMetrics {
+  val metricsGroup = "samza.kafka.producer"
+}
+
+class KafkaSystemProducer(
+  systemName: String,
+  batchSize: Int,
+  reconnectIntervalMs: Long,
+  registry: MetricsRegistry,
+  getProducer: () => Producer[Object, Object]) extends SystemProducer with Logging {
+
+  val flushReconnectCounter = registry.newCounter(KafkaSystemProducerMetrics.metricsGroup, "Producer-%s-Reconnects" format systemName)
+  var sourceBuffers = Map[String, ArrayBuffer[KeyedMessage[Object, Object]]]()
+  var producer: Producer[Object, Object] = null
+
+  def start() {
+  }
+
+  def stop() {
+    if (producer != null) {
+      producer.close
+    }
+  }
+
+  def register(source: String) {
+    sourceBuffers += source -> ArrayBuffer()
+  }
+
+  def send(source: String, envelope: OutgoingMessageEnvelope) {
+    sourceBuffers(source) += new KeyedMessage[Object, Object](
+      envelope.getSystemStream.getStream,
+      envelope.getKey,
+      envelope.getPartitionKey,
+      envelope.getMessage)
+
+    if (sourceBuffers(source).size >= batchSize) {
+      commit(source)
+    }
+  }
+
+  def commit(source: String) {
+    val buffer = sourceBuffers(source)
+    var done = false
+
+    debug("Flushing buffer with size: %s." format buffer.size)
+
+    while (!done) {
+      try {
+        if (producer == null) {
+          info("Creating a new producer for system %s." format systemName)
+          producer = getProducer()
+          debug("Created a new producer for system %s." format systemName)
+        }
+
+        producer.send(buffer: _*)
+        done = true
+      } catch {
+        case e: Throwable =>
+          warn("Triggering a reconnect for %s because connection failed: %s" format (systemName, e.getMessage))
+          debug("Exception while producing to %s." format systemName, e)
+
+          flushReconnectCounter.inc
+
+          if (producer != null) {
+            producer.close
+            producer = null
+          }
+
+          try {
+            Thread.sleep(reconnectIntervalMs)
+          } catch {
+            case e: InterruptedException => None
+          }
+      }
+    }
+
+    buffer.clear
+    debug("Flushed buffer.")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/MessageSink.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/MessageSink.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/MessageSink.scala
new file mode 100644
index 0000000..71fae59
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/MessageSink.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import kafka.common.TopicAndPartition
+import kafka.message.MessageAndOffset
+
+private[kafka] trait MessageSink {
+  def setIsAtHighWatermark(tp: TopicAndPartition, isAtHighWatermark: Boolean): Unit
+
+  def addMessage(tp: TopicAndPartition, msg: MessageAndOffset, highWatermark: Long): Unit
+
+  def abdicate(tp: TopicAndPartition, lastOffset: Long): Unit
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicAndPartitionMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicAndPartitionMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicAndPartitionMetrics.scala
new file mode 100644
index 0000000..75fc022
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicAndPartitionMetrics.scala
@@ -0,0 +1,56 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.metrics.{MetricsRegistry, Counter}
+import kafka.common.TopicAndPartition
+import java.util.concurrent.ConcurrentHashMap
+import grizzled.slf4j.Logging
+
+/**
+ * Wrapper around the metrics that BrokerProxies will be updating per topic partition.  Multiple BrokerProxies will
+ * be updating the map at the same time, but no two BrokerProxies should be updating the same key at the same time.
+ *
+ * @param metricsRegistry Registry to hook counters into.
+ */
+private[kafka] class TopicAndPartitionMetrics(metricsRegistry:MetricsRegistry) extends Logging {
+  val metricsGroup = "KafkaSystem"
+
+  val counters = new ConcurrentHashMap[TopicAndPartition, (Counter,Counter,Counter)]()
+
+  def addNewTopicAndPartition(tp:TopicAndPartition) = {
+    if(containsTopicAndPartition(tp)) {
+      warn("TopicAndPartitionsMetrics already has an entry for topic-partition %s, not adding." format tp)
+    } else {
+      counters.put(tp, (newCounter("%s-OffsetChange" format tp),  newCounter("%s-BytesRead" format tp), newCounter("%s-Reads" format tp)))
+    }
+  }
+
+  def containsTopicAndPartition(tp:TopicAndPartition) = counters.containsKey(tp)
+
+  def newCounter = metricsRegistry.newCounter(metricsGroup, _:String)
+
+  def getOffsetCounter(tp:TopicAndPartition) = counters.get(tp)._1
+
+  def getBytesReadCounter(tp:TopicAndPartition) = counters.get(tp)._2
+
+  def getReadsCounter(tp:TopicAndPartition) = counters.get(tp)._3
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala
new file mode 100644
index 0000000..8a24ce3
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/TopicMetadataCache.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import scala.annotation.implicitNotFound
+
+import grizzled.slf4j.Logging
+import kafka.api.TopicMetadata
+import kafka.common.ErrorMapping
+
+/**
+ * TopicMetadataCache is used to cache all the topic metadata for Kafka per
+ * (system, topic) partition. The cache access is thread safe. Each entry in
+ * the cache is refreshed after a specified interval. The cache uses the passed
+ * in getTopicInfoFromStore that retrieves the topic metadata from the store (usually zookeeper).
+ */
+object TopicMetadataCache extends Logging {
+  private case class MetadataInfo(var streamMetadata: TopicMetadata, var lastRefreshMs: Long)
+  private val topicMetadataMap: scala.collection.mutable.Map[(String, String), MetadataInfo] = new scala.collection.mutable.HashMap[(String, String), MetadataInfo]
+  private val lock = new Object
+
+  // used to fetch the topic metadata from the store. Accepts a topic and system
+  type FetchTopicMetadataType = (Set[String]) => Map[String, TopicMetadata]
+
+  def getTopicMetadata(topics: Set[String], systemName: String, getTopicInfoFromStore: FetchTopicMetadataType, cacheTimeout: Long = 5000L, getTime: () => Long = { System.currentTimeMillis }): Map[String, TopicMetadata] = {
+    lock synchronized {
+      val time = getTime()
+      val missingTopics = topics.filter(topic => !topicMetadataMap.contains(systemName, topic))
+      val topicsWithBadOrExpiredMetadata = (topics -- missingTopics).filter(topic => {
+        val metadata = topicMetadataMap(systemName, topic)
+        metadata.streamMetadata.errorCode != ErrorMapping.NoError || ((time - metadata.lastRefreshMs) > cacheTimeout)
+      })
+      val topicsToRefresh = missingTopics ++ topicsWithBadOrExpiredMetadata
+
+      if (topicsToRefresh.size > 0) {
+        // Refresh topic information for any missing, expired, or bad topic metadata.
+        topicMetadataMap ++= getTopicInfoFromStore(missingTopics ++ topicsWithBadOrExpiredMetadata)
+          .map { case (topic, metadata) => ((systemName, topic), MetadataInfo(metadata, getTime())) }
+          .toMap
+      }
+
+      // Use our new updated cache to return a map of topic -> metadata
+      topicMetadataMap
+        .filterKeys(topics.map(topic => (systemName, topic)))
+        .map {
+          case ((systemName, topic), metadata) =>
+            (topic, metadata.streamMetadata)
+        }.toMap
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/system/kafka/Toss.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/Toss.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/Toss.scala
new file mode 100644
index 0000000..5cda26e
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/Toss.scala
@@ -0,0 +1,28 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system.kafka
+
+import org.apache.samza.SamzaException
+
+private[kafka] trait Toss {
+  def toss(s:String) = throw new SamzaException(s)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/util/ClientUtilTopicMetadataStore.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/util/ClientUtilTopicMetadataStore.scala b/samza-kafka/src/main/scala/org/apache/samza/util/ClientUtilTopicMetadataStore.scala
new file mode 100644
index 0000000..0bc1867
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/util/ClientUtilTopicMetadataStore.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util
+
+import kafka.api.{ TopicMetadataResponse, TopicMetadata }
+import org.apache.samza.SamzaException
+import kafka.client.ClientUtils
+import grizzled.slf4j.Logging
+import kafka.common.ErrorMapping
+import kafka.cluster.Broker
+import java.util.concurrent.atomic.AtomicInteger
+
+trait TopicMetadataStore extends Logging {
+  def getTopicInfo(topics: Set[String]): Map[String, TopicMetadata]
+}
+
+class ClientUtilTopicMetadataStore(brokersListString: String, clientId: String, timeout: Int = 6000) extends TopicMetadataStore {
+  val brokers = ClientUtils.parseBrokerList(brokersListString)
+  var corrID = new AtomicInteger(0)
+
+  def getTopicInfo(topics: Set[String]) = {
+    val currCorrId = corrID.getAndIncrement
+    val response: TopicMetadataResponse = ClientUtils.fetchTopicMetadata(topics, brokers, clientId, timeout, currCorrId)
+
+    if (response.correlationId != currCorrId) {
+      throw new SamzaException("CorrelationID did not match for request on topics %s (sent %d, got %d)" format (topics, currCorrId, response.correlationId))
+    }
+
+    response.topicsMetadata
+      .map(metadata => (metadata.topic, metadata))
+      .toMap
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
new file mode 100644
index 0000000..d660b91
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util
+
+import org.apache.samza.config.{KafkaConfig, Config, ConfigException}
+import org.apache.samza.config.JobConfig.Config2Job
+import java.util.concurrent.atomic.AtomicLong
+import kafka.client.ClientUtils
+import org.apache.samza.SamzaException
+
+object KafkaUtil {
+  val counter = new AtomicLong(0)
+
+  def getClientId(id: String, config: Config): String = getClientId(
+    id,
+    config.getName.getOrElse(throw new ConfigException("Missing job name.")),
+    config.getJobId.getOrElse("1"))
+
+  def getClientId(id: String, jobName: String, jobId: String): String =
+    "%s-%s-%s-%s-%s" format
+      (id.replaceAll("[^A-Za-z0-9]", "_"),
+        jobName.replaceAll("[^A-Za-z0-9]", "_"),
+        jobId.replaceAll("[^A-Za-z0-9]", "_"),
+        System.currentTimeMillis,
+        counter.getAndIncrement)
+
+}


[04/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-shell/src/main/bash/run-task.sh
----------------------------------------------------------------------
diff --git a/samza-shell/src/main/bash/run-task.sh b/samza-shell/src/main/bash/run-task.sh
new file mode 100755
index 0000000..72cee18
--- /dev/null
+++ b/samza-shell/src/main/bash/run-task.sh
@@ -0,0 +1,19 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+exec $(dirname $0)/run-class.sh org.apache.samza.container.SamzaContainer $@

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/java.hprof.txt
----------------------------------------------------------------------
diff --git a/samza-test/java.hprof.txt b/samza-test/java.hprof.txt
new file mode 100644
index 0000000..c5693d2
--- /dev/null
+++ b/samza-test/java.hprof.txt
@@ -0,0 +1,65 @@
+JAVA PROFILE 1.0.1, created Sat Aug  3 14:27:50 2013
+
+Header for -agentlib:hprof (or -Xrunhprof) ASCII Output (JDK 5.0 JVMTI based)
+
+%W% %E%
+
+ Copyright (c) 2006 Sun Microsystems, Inc. All  Rights Reserved.
+
+WARNING!  This file format is under development, and is subject to
+change without notice.
+
+This file contains the following types of records:
+
+THREAD START
+THREAD END      mark the lifetime of Java threads
+
+TRACE           represents a Java stack trace.  Each trace consists
+                of a series of stack frames.  Other records refer to
+                TRACEs to identify (1) where object allocations have
+                taken place, (2) the frames in which GC roots were
+                found, and (3) frequently executed methods.
+
+HEAP DUMP       is a complete snapshot of all live objects in the Java
+                heap.  Following distinctions are made:
+
+                ROOT    root set as determined by GC
+                CLS     classes 
+                OBJ     instances
+                ARR     arrays
+
+SITES           is a sorted list of allocation sites.  This identifies
+                the most heavily allocated object types, and the TRACE
+                at which those allocations occurred.
+
+CPU SAMPLES     is a statistical profile of program execution.  The VM
+                periodically samples all running threads, and assigns
+                a quantum to active TRACEs in those threads.  Entries
+                in this record are TRACEs ranked by the percentage of
+                total quanta they consumed; top-ranked TRACEs are
+                typically hot spots in the program.
+
+CPU TIME        is a profile of program execution obtained by measuring
+                the time spent in individual methods (excluding the time
+                spent in callees), as well as by counting the number of
+                times each method is called. Entries in this record are
+                TRACEs ranked by the percentage of total CPU time. The
+                "count" field indicates the number of times each TRACE 
+                is invoked.
+
+MONITOR TIME    is a profile of monitor contention obtained by measuring
+                the time spent by a thread waiting to enter a monitor.
+                Entries in this record are TRACEs ranked by the percentage
+                of total monitor contention time and a brief description
+                of the monitor.  The "count" field indicates the number of 
+                times the monitor was contended at that TRACE.
+
+MONITOR DUMP    is a complete snapshot of all the monitors and threads in 
+                the System.
+
+HEAP DUMP, SITES, CPU SAMPLES|TIME and MONITOR DUMP|TIME records are generated 
+at program exit.  They can also be obtained during program execution by typing 
+Ctrl-\ (on Solaris) or by typing Ctrl-Break (on Win32).
+
+--------
+

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/SimpleStatefulTask.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/SimpleStatefulTask.java b/samza-test/src/main/java/org/apache/samza/test/integration/SimpleStatefulTask.java
new file mode 100644
index 0000000..973b0ba
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/SimpleStatefulTask.java
@@ -0,0 +1,35 @@
+package org.apache.samza.test.integration;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.storage.kv.Entry;
+import org.apache.samza.storage.kv.KeyValueIterator;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+public class SimpleStatefulTask implements StreamTask, InitableTask {
+  
+  private KeyValueStore<String, String> store;
+  
+  public void init(Config config, TaskContext context) {
+    this.store = (KeyValueStore<String, String>) context.getStore("mystore");
+    System.out.println("Contents of store: ");
+    KeyValueIterator<String, String> iter = store.all();
+    while(iter.hasNext()) {
+      Entry<String, String> entry = iter.next();
+      System.out.println(entry.getKey() + " => " + entry.getValue());
+    }
+    iter.close();
+  }
+  
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    System.out.println("Adding " + envelope.getMessage() + " => " + envelope.getMessage() + " to the store.");
+    store.put((String) envelope.getMessage(), (String) envelope.getMessage());
+    coordinator.commit();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/StatePerfTestTask.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/StatePerfTestTask.java b/samza-test/src/main/java/org/apache/samza/test/integration/StatePerfTestTask.java
new file mode 100644
index 0000000..873de74
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/StatePerfTestTask.java
@@ -0,0 +1,35 @@
+package org.apache.samza.test.integration;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+public class StatePerfTestTask implements StreamTask, InitableTask {
+  
+  private KeyValueStore<String, String> store;
+  private int count = 0;
+  private int LOG_INTERVAL = 100000;
+  private long start = System.currentTimeMillis();
+  
+  public void init(Config config, TaskContext context) {
+    this.store = (KeyValueStore<String, String>) context.getStore("mystore");
+  }
+  
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    store.put((String) envelope.getMessage(), (String) envelope.getMessage());
+    count++;
+    if(count % LOG_INTERVAL == 0) {
+      double ellapsedSecs = (System.currentTimeMillis() - start)/1000.0;
+      System.out.println(String.format("Throughput = %.2f messages/sec.", count/ellapsedSecs));
+      start = System.currentTimeMillis();
+      count = 0;
+      coordinator.commit();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/join/Checker.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/join/Checker.java b/samza-test/src/main/java/org/apache/samza/test/integration/join/Checker.java
new file mode 100644
index 0000000..22f5e87
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/join/Checker.java
@@ -0,0 +1,71 @@
+package org.apache.samza.test.integration.join;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.storage.kv.KeyValueIterator;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.task.WindowableTask;
+import static java.lang.System.out;
+
+public class Checker implements StreamTask, WindowableTask, InitableTask {
+
+  private static String CURRENT_EPOCH = "current-epoch";
+  private KeyValueStore<String, String> store;
+  private int expectedKeys;
+  private int numPartitions;
+  
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.store = (KeyValueStore<String, String>) context.getStore("checker-state");
+    this.expectedKeys = config.getInt("expected.keys");
+    this.numPartitions = config.getInt("num.partitions");
+  }
+
+  @Override
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    String key = (String) envelope.getKey();
+    String epoch = (String) envelope.getMessage();
+    checkEpoch(epoch);
+    this.store.put(key, epoch);
+  }
+  
+  @Override
+  public void window(MessageCollector collector, TaskCoordinator coordinator) {
+    KeyValueIterator<String, String> iter = this.store.all();
+    String currentEpoch = this.store.get(CURRENT_EPOCH);
+    out.println("Checking if epoch " + currentEpoch + " is complete.");
+    int count = 0;
+    while(iter.hasNext()) {
+      String foundEpoch = iter.next().getValue();
+      if(foundEpoch.equals(currentEpoch))
+        count += 1;
+    }
+    iter.close();
+    if(count == expectedKeys + 1) {
+      out.println("Epoch " + currentEpoch + " is complete.");
+      int nextEpoch = Integer.parseInt(currentEpoch) + 1;
+      for(int i = 0; i < numPartitions; i++)
+        collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "epoch"), i, Integer.toString(nextEpoch))); 
+    } else if(count > expectedKeys + 1) {
+      throw new IllegalStateException("Got " + count + " keys, which is more than the expected " + (expectedKeys + 1));
+    } else {
+      out.println("Only found " + count + " valid keys, try again later.");
+    }
+  }
+  
+  private void checkEpoch(String epoch) {
+    String curr = this.store.get(CURRENT_EPOCH);
+    if(curr == null)
+      this.store.put(CURRENT_EPOCH, epoch);
+    else if(!curr.equals(epoch))
+      throw new IllegalArgumentException("Got epoch " + epoch + " but have not yet completed " + curr);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/join/Emitter.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/join/Emitter.java b/samza-test/src/main/java/org/apache/samza/test/integration/join/Emitter.java
new file mode 100644
index 0000000..2989ca7
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/join/Emitter.java
@@ -0,0 +1,88 @@
+package org.apache.samza.test.integration.join;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.task.WindowableTask;
+
+/**
+ * This job takes input from "epoch" and for each epoch emits "max" records of form
+ *   (key = counter, value = epoch-partition)
+ *   
+ */
+@SuppressWarnings("unchecked")
+public class Emitter implements StreamTask, InitableTask, WindowableTask {
+  
+  private static String EPOCH = "the-epoch";
+  private static String COUNT = "the-count";
+  
+  private KeyValueStore<String, String> state;
+  private int max;
+  private String partition;
+
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.state = (KeyValueStore<String, String>) context.getStore("emitter-state");
+    this.partition = Integer.toString(context.getPartition().getPartitionId());
+    this.max = config.getInt("count");
+  }
+
+  @Override
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    if(envelope.getSystemStreamPartition().getStream().equals("epoch")) {
+      int newEpoch = Integer.parseInt((String) envelope.getMessage());
+      Integer epoch = getInt(EPOCH);
+      if(epoch == null || newEpoch == epoch)
+        return;
+      if(newEpoch < epoch)
+        throw new IllegalArgumentException("Got new epoch " + newEpoch + " which is less than current epoch " + epoch);
+      
+      // it's a new era, reset current epoch and count
+      this.state.put(EPOCH, Integer.toString(epoch));
+      this.state.put(COUNT, "0");
+      coordinator.commit();
+    }
+  }
+  
+  public void window(MessageCollector collector, TaskCoordinator coordinator) {
+    Integer epoch = getInt(EPOCH);
+    if(epoch == null) {
+      resetEpoch();
+      return;
+    }
+    int counter = getInt(COUNT);
+    if(counter < max) {
+      OutgoingMessageEnvelope envelope = new OutgoingMessageEnvelope(new SystemStream("kafka", "emitted"), Integer.toString(counter), epoch + "-" + partition);
+      collector.send(envelope);
+      this.state.put(COUNT, Integer.toString(getInt(COUNT) + 1));
+    } else {
+      trySleep(100);
+    }
+  }
+  
+  private void resetEpoch() {
+    state.put(EPOCH, "0");
+    state.put(COUNT, "0");
+  }
+  
+  private Integer getInt(String key) {
+    String value = this.state.get(key);
+    return value == null? null : Integer.parseInt(value);
+  }
+  
+  private void trySleep(long ms) {
+    try {
+      Thread.sleep(ms);
+    } catch(Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/join/EpochPartitioner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/join/EpochPartitioner.java b/samza-test/src/main/java/org/apache/samza/test/integration/join/EpochPartitioner.java
new file mode 100644
index 0000000..d11d300
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/join/EpochPartitioner.java
@@ -0,0 +1,13 @@
+package org.apache.samza.test.integration.join;
+
+import kafka.producer.Partitioner;
+import kafka.utils.VerifiableProperties;
+
+public class EpochPartitioner implements Partitioner {
+  
+  public EpochPartitioner(VerifiableProperties p){}
+  
+  public int partition(Object key, int numParts) {
+    return Integer.parseInt((String) key);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/join/Joiner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/join/Joiner.java b/samza-test/src/main/java/org/apache/samza/test/integration/join/Joiner.java
new file mode 100644
index 0000000..ca8fed4
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/join/Joiner.java
@@ -0,0 +1,93 @@
+package org.apache.samza.test.integration.join;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+@SuppressWarnings("unchecked")
+public class Joiner implements StreamTask, InitableTask {
+  
+  private KeyValueStore<String, String> store;
+  private int expected;
+
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.store = (KeyValueStore<String, String>) context.getStore("joiner-state");
+    this.expected = config.getInt("num.partitions");
+  }
+
+  @Override
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    String key = (String) envelope.getKey();
+    String value = (String) envelope.getMessage();
+    String[] pieces = value.split("-");
+    int epoch = Integer.parseInt(pieces[0]);
+    int partition = Integer.parseInt(pieces[1]);
+    Partitions partitions = loadPartitions(epoch, key);
+    if(partitions.epoch != epoch) {
+      // we are in a new era
+      if(partitions.partitions.size() != expected)
+        throw new IllegalArgumentException("Should have " + expected + " partitions when new epoch starts.");
+      this.store.delete(key);
+      partitions.epoch = epoch;
+      partitions.partitions.clear();
+      partitions.partitions.add(partition);
+    } else {
+      partitions.partitions.add(partition);
+      if(partitions.partitions.size() == expected)
+        collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "completed-keys"), key, Integer.toString(epoch)));
+    }
+    this.store.put(key, partitions.toString());
+  }
+  
+  private Partitions loadPartitions(int epoch, String key) {
+    String current = this.store.get(key);
+    Partitions partitions;
+    if(current == null)
+      partitions = new Partitions(epoch, new HashSet<Integer>());
+    else
+      partitions = Partitions.parse(current);
+    return partitions;
+  }
+  
+  private static class Partitions {
+    int epoch;
+    Set<Integer> partitions;
+    
+    public Partitions(int epoch, Set<Integer> partitions) {
+      this.epoch = epoch;
+      this.partitions = partitions;
+    }
+    
+    public static Partitions parse(String s) {
+      String[] pieces = s.split("\\|", -1);
+      int epoch = Integer.parseInt(pieces[1]);
+      Set<Integer> set = new HashSet<Integer>(pieces.length);
+      for(int i = 2; i < pieces.length - 1; i++)
+        set.add(Integer.parseInt(pieces[i]));
+      return new Partitions(epoch, set);
+    }
+    
+    public String toString() {
+      StringBuilder b = new StringBuilder("|");
+      b.append(epoch);
+      b.append("|");
+      for(int p: partitions) {
+        b.append(p);
+        b.append("|");
+      }
+      return b.toString();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/java/org/apache/samza/test/integration/join/Watcher.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/integration/join/Watcher.java b/samza-test/src/main/java/org/apache/samza/test/integration/join/Watcher.java
new file mode 100644
index 0000000..fac4ee1
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/integration/join/Watcher.java
@@ -0,0 +1,77 @@
+package org.apache.samza.test.integration.join;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Properties;
+
+import javax.mail.Message;
+import javax.mail.MessagingException;
+import javax.mail.Session;
+import javax.mail.Transport;
+import javax.mail.internet.InternetAddress;
+import javax.mail.internet.MimeMessage;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.task.WindowableTask;
+
+public class Watcher implements StreamTask, WindowableTask, InitableTask {
+
+  private boolean inError = false;
+  private long lastEpochChange = System.currentTimeMillis();
+  private long maxTimeBetweenEpochsMs;
+  private int currentEpoch = 0;
+  private String smtpHost;
+  private String to;
+  private String from;
+  
+  @Override
+  public void init(Config config, TaskContext context) {
+    this.maxTimeBetweenEpochsMs = config.getLong("max.time.between.epochs.ms");
+    this.smtpHost = config.get("mail.smtp.host");
+    this.to = config.get("mail.to");
+    this.from = config.get("mail.from");
+  }
+  
+  @Override
+  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
+    int epoch = Integer.parseInt((String) envelope.getMessage());
+    if(epoch > currentEpoch) {
+      this.currentEpoch = epoch;
+      this.lastEpochChange = System.currentTimeMillis();
+      this.inError = false;
+    }
+  }
+  
+  @Override
+  public void window(MessageCollector collector, TaskCoordinator coordinator) {
+    boolean isLagging = System.currentTimeMillis() - lastEpochChange > maxTimeBetweenEpochsMs;
+    if(!inError && isLagging) {
+      this.inError = true;
+      sendEmail(from, to, "Job failed to make progress!", String.format("No epoch change for %d minutes.", this.maxTimeBetweenEpochsMs / (60*1000)));
+    }
+  }
+  
+  private void sendEmail(String from, String to, String subject, String body) {
+    Properties props = new Properties();
+    props.put("mail.smtp.host", smtpHost);
+    Session session = Session.getInstance(props, null);
+    try {
+        MimeMessage msg = new MimeMessage(session);
+        msg.setFrom(new InternetAddress(from));
+        msg.addRecipients(Message.RecipientType.TO, InternetAddress.parse(to));
+        msg.setSubject(subject);
+        msg.setSentDate(new Date());
+        msg.setText(body);
+        Transport.send(msg);
+    } catch (MessagingException e) {
+        throw new RuntimeException(e);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/common.properties
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/common.properties b/samza-test/src/main/resources/common.properties
new file mode 100644
index 0000000..971a219
--- /dev/null
+++ b/samza-test/src/main/resources/common.properties
@@ -0,0 +1,25 @@
+job.factory.class=samza.job.local.LocalJobFactory
+
+task.checkpoint.factory=samza.checkpoint.kafka.KafkaCheckpointManagerFactory
+task.checkpoint.system=kafka-checkpoints
+task.checkpoint.replication.factor=1
+
+serializers.registry.string.class=samza.serializers.StringSerdeFactory
+
+# Kafka System
+systems.kafka.samza.factory=samza.system.kafka.KafkaSystemFactory
+systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager
+systems.kafka.consumer.zookeeper.connect=localhost:2181
+systems.kafka.consumer.auto.offset.reset=smallest
+systems.kafka.producer.metadata.broker.list=localhost:9092
+systems.kafka.samza.msg.serde=string
+
+# Checkpoints System
+systems.kafka-checkpoints.samza.factory=samza.system.kafka.KafkaSystemFactory
+systems.kafka-checkpoints.serializer.class=samza.task.state.KafkaCheckpointEncoder
+systems.kafka-checkpoints.partitioner.class=samza.task.state.KafkaCheckpointPartitioner
+systems.kafka-checkpoints.key.serializer.class=kafka.serializer.NullEncoder
+systems.kafka-checkpoints.producer.metadata.broker.list=localhost:9092
+systems.kafka-checkpoints.consumer.zookeeper.connect=localhost:2181
+systems.kafka-checkpoints.producer.type=sync
+

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/hello-stateful-world.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/hello-stateful-world.samsa b/samza-test/src/main/resources/hello-stateful-world.samsa
new file mode 100644
index 0000000..84325d0
--- /dev/null
+++ b/samza-test/src/main/resources/hello-stateful-world.samsa
@@ -0,0 +1,13 @@
+# Job
+job.factory.class=samza.job.local.LocalJobFactory
+job.name=hello-stateful-world
+
+# Task
+task.class=samza.test.integration.SimpleStatefulTask
+task.inputs=kafka.input
+
+# Stores
+stores.mystore.factory=samza.storage.kv.KeyValueStorageEngineFactory
+stores.mystore.key.serde=string
+stores.mystore.msg.serde=string
+stores.mystore.changelog=kafka.mystore

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/join/checker.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/join/checker.samsa b/samza-test/src/main/resources/join/checker.samsa
new file mode 100644
index 0000000..e41ffa0
--- /dev/null
+++ b/samza-test/src/main/resources/join/checker.samsa
@@ -0,0 +1,18 @@
+# Job
+job.name=checker
+
+systems.kafka.partitioner.class=samza.test.integration.join.EpochPartitioner
+
+# Task
+task.class=samza.test.integration.join.Checker
+task.inputs=kafka.completed-keys
+
+stores.checker-state.factory=samza.storage.kv.KeyValueStorageEngineFactory
+stores.checker-state.key.serde=string
+stores.checker-state.msg.serde=string
+stores.checker-state.changelog=kafka.checker-state
+
+task.window.ms=300000
+
+num.partitions=4
+expected.keys=100000

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/join/emitter.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/join/emitter.samsa b/samza-test/src/main/resources/join/emitter.samsa
new file mode 100644
index 0000000..140d13d
--- /dev/null
+++ b/samza-test/src/main/resources/join/emitter.samsa
@@ -0,0 +1,15 @@
+# Job
+job.name=emitter
+
+# Task
+task.class=samza.test.integration.join.Emitter
+task.inputs=kafka.epoch
+
+stores.emitter-state.factory=samza.storage.kv.KeyValueStorageEngineFactory
+stores.emitter-state.key.serde=string
+stores.emitter-state.msg.serde=string
+stores.emitter-state.changelog=kafka.emitter-state
+
+task.window.ms=0
+
+count=100000

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/join/joiner.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/join/joiner.samsa b/samza-test/src/main/resources/join/joiner.samsa
new file mode 100644
index 0000000..27655d8
--- /dev/null
+++ b/samza-test/src/main/resources/join/joiner.samsa
@@ -0,0 +1,13 @@
+# Job
+job.name=joiner
+
+# Task
+task.class=samza.test.integration.join.Joiner
+task.inputs=kafka.emitted
+
+stores.joiner-state.factory=samza.storage.kv.KeyValueStorageEngineFactory
+stores.joiner-state.key.serde=string
+stores.joiner-state.msg.serde=string
+stores.joiner-state.changelog=kafka.checker-state
+
+num.partitions=4

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/join/watcher.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/join/watcher.samsa b/samza-test/src/main/resources/join/watcher.samsa
new file mode 100644
index 0000000..1a64ebf
--- /dev/null
+++ b/samza-test/src/main/resources/join/watcher.samsa
@@ -0,0 +1,13 @@
+# Job
+job.name=watcher
+
+# Task
+task.class=samza.test.integration.join.Joiner
+task.inputs=kafka.epoch
+
+task.window.ms=300000
+
+max.time.between.epochs.ms=600000
+mail.smtp.host=email.corp.linkedin.com
+mail.to=samza-eng@linkedin.com
+mail.from=gregor@incubator.apache.org

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/log4j.xml b/samza-test/src/main/resources/log4j.xml
new file mode 100644
index 0000000..ecaf8a2
--- /dev/null
+++ b/samza-test/src/main/resources/log4j.xml
@@ -0,0 +1,21 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+  <appender name="console" class="org.apache.log4j.ConsoleAppender">
+    <param name="Target" value="System.out" />
+    <layout class="org.apache.log4j.PatternLayout">
+      <param name="ConversionPattern" value="%d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n" />
+    </layout>
+  </appender>
+
+  <logger name="org.apache.hadoop">
+    <level value="off"/>
+  </logger>
+
+  <root>
+    <priority value="info" />
+    <appender-ref ref="console" />
+  </root>
+
+</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/main/resources/perf/counter.samsa
----------------------------------------------------------------------
diff --git a/samza-test/src/main/resources/perf/counter.samsa b/samza-test/src/main/resources/perf/counter.samsa
new file mode 100644
index 0000000..cf06c9e
--- /dev/null
+++ b/samza-test/src/main/resources/perf/counter.samsa
@@ -0,0 +1,12 @@
+# Job
+job.name=counter-task
+
+# Task
+task.class=samza.test.integration.StatePerfTestTask
+task.inputs=kafka.input
+
+# Stores
+stores.mystore.factory=samza.storage.kv.KeyValueStorageEngineFactory
+stores.mystore.key.serde=string
+stores.mystore.msg.serde=string
+stores.mystore.changelog=kafka.mystore

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
----------------------------------------------------------------------
diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
new file mode 100644
index 0000000..7d4e20a
--- /dev/null
+++ b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
@@ -0,0 +1,436 @@
+package org.apache.samza.test.integration
+
+import org.apache.samza.task.StreamTask
+import org.apache.samza.task.TaskContext
+import org.apache.samza.task.InitableTask
+import org.apache.samza.config.Config
+import scala.collection.JavaConversions._
+import org.apache.samza.task.TaskCoordinator
+import org.apache.samza.task.MessageCollector
+import org.apache.samza.system.IncomingMessageEnvelope
+import org.apache.samza.checkpoint.Checkpoint
+import org.junit.BeforeClass
+import org.junit.AfterClass
+import kafka.zk.EmbeddedZookeeper
+import kafka.utils.TestUtils
+import org.apache.samza.system.SystemStream
+import kafka.utils.TestZKUtils
+import kafka.server.KafkaConfig
+import org.I0Itec.zkclient.ZkClient
+import kafka.producer.ProducerConfig
+import kafka.server.KafkaServer
+import kafka.utils.Utils
+import org.apache.samza.storage.kv.KeyValueStore
+import org.apache.samza.util._
+import org.junit.Test
+import kafka.admin.AdminUtils
+import kafka.common.ErrorMapping
+import org.junit.Assert._
+import kafka.utils.ZKStringSerializer
+import scala.collection.mutable.ArrayBuffer
+import org.apache.samza.job.local.LocalJobFactory
+import org.apache.samza.job.ApplicationStatus
+import java.util.concurrent.CountDownLatch
+import org.apache.samza.job.local.ThreadJob
+import org.apache.samza.util.TopicMetadataStore
+import org.apache.samza.util.ClientUtilTopicMetadataStore
+import org.apache.samza.config.MapConfig
+import org.apache.samza.system.kafka.TopicMetadataCache
+import org.apache.samza.container.SamzaContainer
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.SynchronizedMap
+import org.apache.samza.Partition
+import java.util.concurrent.TimeUnit
+import kafka.producer.Producer
+import kafka.producer.KeyedMessage
+import java.util.concurrent.Semaphore
+import java.util.concurrent.CyclicBarrier
+import kafka.consumer.Consumer
+import kafka.consumer.ConsumerConfig
+import java.util.Properties
+import java.util.concurrent.Executors
+import kafka.message.MessageAndOffset
+import kafka.message.MessageAndMetadata
+import org.apache.samza.job.StreamJob
+
+object TestStatefulTask {
+  val INPUT_TOPIC = "input"
+  val STATE_TOPIC = "mystore"
+  val TOTAL_PARTITIONS = 1
+  val REPLICATION_FACTOR = 3
+
+  val zkConnect: String = TestZKUtils.zookeeperConnect
+  var zkClient: ZkClient = null
+  val zkConnectionTimeout = 6000
+  val zkSessionTimeout = 6000
+
+  val brokerId1 = 0
+  val brokerId2 = 1
+  val brokerId3 = 2
+  val ports = TestUtils.choosePorts(3)
+  val (port1, port2, port3) = (ports(0), ports(1), ports(2))
+
+  val props1 = TestUtils.createBrokerConfig(brokerId1, port1)
+  val config1 = new KafkaConfig(props1) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+    override val zkConnect = TestStatefulTask.zkConnect + "/"
+  }
+  val props2 = TestUtils.createBrokerConfig(brokerId2, port2)
+  val config2 = new KafkaConfig(props2) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+    override val zkConnect = TestStatefulTask.zkConnect + "/"
+  }
+  val props3 = TestUtils.createBrokerConfig(brokerId3, port3)
+  val config3 = new KafkaConfig(props3) {
+    override val hostName = "localhost"
+    override val numPartitions = 1
+    override val zkConnect = TestStatefulTask.zkConnect + "/"
+  }
+
+  val config = new java.util.Properties()
+  val brokers = "localhost:%d,localhost:%d,localhost:%d" format (port1, port2, port3)
+  config.put("metadata.broker.list", brokers)
+  config.put("producer.type", "sync")
+  config.put("request.required.acks", "-1")
+  config.put("serializer.class", "kafka.serializer.StringEncoder");
+  val producerConfig = new ProducerConfig(config)
+  var producer: Producer[String, String] = null
+  val cp1 = new Checkpoint(Map(new SystemStream("kafka", "topic") -> "123"))
+  val cp2 = new Checkpoint(Map(new SystemStream("kafka", "topic") -> "12345"))
+  var zookeeper: EmbeddedZookeeper = null
+  var server1: KafkaServer = null
+  var server2: KafkaServer = null
+  var server3: KafkaServer = null
+  var metadataStore: TopicMetadataStore = null
+
+  @BeforeClass
+  def beforeSetupServers {
+    zookeeper = new EmbeddedZookeeper(zkConnect)
+    server1 = TestUtils.createServer(config1)
+    server2 = TestUtils.createServer(config2)
+    server3 = TestUtils.createServer(config3)
+    zkClient = new ZkClient(zkConnect + "/", 6000, 6000, ZKStringSerializer)
+    producer = new Producer(producerConfig)
+    metadataStore = new ClientUtilTopicMetadataStore(brokers, "some-job-name")
+
+    createTopics
+    validateTopics
+  }
+
+  def createTopics {
+    AdminUtils.createTopic(
+      zkClient,
+      INPUT_TOPIC,
+      TOTAL_PARTITIONS,
+      REPLICATION_FACTOR)
+
+    AdminUtils.createTopic(
+      zkClient,
+      STATE_TOPIC,
+      TOTAL_PARTITIONS,
+      REPLICATION_FACTOR)
+  }
+
+  def validateTopics {
+    val topics = Set(STATE_TOPIC, INPUT_TOPIC)
+    var done = false
+    var retries = 0
+
+    while (!done && retries < 100) {
+      try {
+        val topicMetadataMap = TopicMetadataCache.getTopicMetadata(topics, "kafka", metadataStore.getTopicInfo)
+
+        topics.foreach(topic => {
+          val topicMetadata = topicMetadataMap(topic)
+          val errorCode = topicMetadata.errorCode
+
+          ErrorMapping.maybeThrowException(errorCode)
+        })
+
+        done = true
+      } catch {
+        case e: Throwable =>
+          System.err.println("Got exception while validating test topics. Waiting and retrying.", e)
+          retries += 1
+          Thread.sleep(500)
+      }
+    }
+
+    if (retries >= 100) {
+      fail("Unable to successfully create topics. Tried to validate %s times." format retries)
+    }
+  }
+
+  @AfterClass
+  def afterCleanLogDirs {
+    server1.shutdown
+    server1.awaitShutdown()
+    server2.shutdown
+    server2.awaitShutdown()
+    server3.shutdown
+    server3.awaitShutdown()
+    Utils.rm(server1.config.logDirs)
+    Utils.rm(server2.config.logDirs)
+    Utils.rm(server3.config.logDirs)
+    zkClient.close
+  }
+}
+
+/**
+ * Test that does the following:
+ * 
+ * 1. Starts ZK, and 3 kafka brokers.
+ * 2. Create two topics: input and mystore.
+ * 3. Validate that the topics were created successfully and have leaders.
+ * 4. Start a single partition of TestTask using LocalJobFactory/ThreadJob.
+ * 5. Send four messages to input (1,2,3,2), which contain one dupe (2).
+ * 6. Validate that all messages were received by TestTask.
+ * 7. Validate that TestTask called store.put() for all four messages, and that the messages ended up in the mystore topic.
+ * 8. Kill the job.
+ * 9. Start the job again.
+ * 10. Validate that the job restored all messages (1,2,3) to the store.
+ * 11. Send three more messages to input (4,5,5), and validate that TestTask receives them.
+ * 12. Kill the job again.
+ */
+class TestStatefulTask {
+  import TestStatefulTask._
+
+  val jobConfig = Map(
+    "job.factory.class" -> "org.apache.samza.job.local.LocalJobFactory",
+    "job.name" -> "hello-stateful-world",
+    "task.class" -> "org.apache.samza.test.integration.TestTask",
+    "task.inputs" -> "kafka.input",
+    "serializers.registry.string.class" -> "org.apache.samza.serializers.StringSerdeFactory",
+    "stores.mystore.factory" -> "org.apache.samza.storage.kv.KeyValueStorageEngineFactory",
+    "stores.mystore.key.serde" -> "string",
+    "stores.mystore.msg.serde" -> "string",
+    "stores.mystore.changelog" -> "kafka.mystore",
+    "systems.kafka.samza.factory" -> "org.apache.samza.system.kafka.KafkaSystemFactory",
+    "systems.kafka.samza.partition.manager" -> "org.apache.samza.stream.kafka.KafkaPartitionManager",
+    "systems.kafka.consumer.zookeeper.connect" -> zkConnect,
+    "systems.kafka.consumer.auto.offset.reset" -> "smallest",
+    "systems.kafka.producer.metadata.broker.list" -> ("localhost:%s" format port1),
+    "systems.kafka.samza.msg.serde" -> "string")
+
+  @Test
+  def testShouldStartAndRestore {
+    // Have to do this in one test to guarantee ordering.
+    testShouldStartTaskForFirstTime
+    testShouldRestoreStore
+  }
+
+  def testShouldStartTaskForFirstTime {
+    val (job, task) = startJob
+
+    // Validate that restored is empty.
+    assertEquals(0, task.initFinished.getCount)
+    assertEquals(0, task.restored.size)
+    assertEquals(0, task.received.size)
+
+    // Send some messages to input stream.
+    send(task, "1")
+    send(task, "2")
+    send(task, "3")
+    send(task, "2")
+
+    // Validate that messages appear in store stream.
+    val messages = readAll(STATE_TOPIC, 3, "testShouldStartTaskForFirstTime")
+
+    assertEquals(4, messages.length)
+    assertEquals("1", messages(0))
+    assertEquals("2", messages(1))
+    assertEquals("3", messages(2))
+    assertEquals("2", messages(3))
+
+    stopJob(job)
+  }
+
+  def testShouldRestoreStore {
+    val (job, task) = startJob
+
+    // Validate that restored is empty.
+    assertEquals(3, task.restored.size)
+    assertTrue(task.restored.contains("1"))
+    assertTrue(task.restored.contains("2"))
+    assertTrue(task.restored.contains("3"))
+
+    var count = 0
+
+    // We should get the original four messages in the stream (1,2,3,2).
+    // Note that this will trigger four new outgoing messages to the STATE_TOPIC.
+    while (task.received.size < 4 && count < 100) {
+      Thread.sleep(600)
+      count += 1
+    }
+
+    assertTrue(count < 100)
+
+    // Reset the count down latch after the 4 messages come in.
+    task.awaitMessage
+
+    // Send some messages to input stream.
+    send(task, "4")
+    send(task, "5")
+    send(task, "5")
+
+    // Validate that messages appear in store stream.
+    val messages = readAll(STATE_TOPIC, 10, "testShouldRestoreStore")
+
+    assertEquals(11, messages.length)
+    // From initial start.
+    assertEquals("1", messages(0))
+    assertEquals("2", messages(1))
+    assertEquals("3", messages(2))
+    assertEquals("2", messages(3))
+    // From second start, since we're we have no checkpoint manager defined, 
+    // and are starting back at offset 0 for input stream, again.
+    assertEquals("1", messages(4))
+    assertEquals("2", messages(5))
+    assertEquals("3", messages(6))
+    assertEquals("2", messages(7))
+    // From sending in this method.
+    assertEquals("4", messages(8))
+    assertEquals("5", messages(9))
+    assertEquals("5", messages(10))
+
+    stopJob(job)
+  }
+
+  /**
+   * Start a job for TestJob, and do some basic sanity checks around startup 
+   * time, number of partitions, etc.
+   */
+  def startJob = {
+    val jobFactory = new LocalJobFactory
+    val job = jobFactory.getJob(new MapConfig(jobConfig))
+
+    // Start task.
+    job.submit
+    assertEquals(ApplicationStatus.Running, job.waitForStatus(ApplicationStatus.Running, 60000))
+    TestTask.awaitTaskRegistered
+    val tasks = TestTask.tasks
+
+    // Should only have one partition.
+    assertEquals(1, tasks.size)
+
+    val task = tasks.values.toList.head
+
+    task.initFinished.await(60, TimeUnit.SECONDS)
+    assertEquals(0, task.initFinished.getCount)
+
+    (job, task)
+  }
+
+  /**
+   * Kill a job, and wait for an unsuccessful finish (since this throws an 
+   * interrupt, which is forwarded on to ThreadJob, and marked as a failure).
+   */
+  def stopJob(job: StreamJob) {
+    // Shutdown task.
+    job.kill
+    assertEquals(ApplicationStatus.UnsuccessfulFinish, job.waitForFinish(60000))
+  }
+
+  /**
+   * Send a message to the input topic, and validate that it gets to the test task.
+   */
+  def send(task: TestTask, msg: String) {
+    producer.send(new KeyedMessage(INPUT_TOPIC, msg))
+    task.awaitMessage
+    assertEquals(msg, task.received.last)
+  }
+
+  /**
+   * Read all messages from a topic starting from last saved offset for group. 
+   * To read all from offset 0, specify a unique, new group string.
+   */
+  def readAll(topic: String, maxOffsetInclusive: Int, group: String): List[String] = {
+    val props = new Properties
+
+    props.put("zookeeper.connect", zkConnect)
+    props.put("group.id", group)
+    props.put("auto.offset.reset", "smallest")
+
+    val consumerConfig = new ConsumerConfig(props)
+    val consumerConnector = Consumer.create(consumerConfig)
+    var stream = consumerConnector.createMessageStreams(Map(topic -> 1)).get(topic).get.get(0).iterator
+    var message: MessageAndMetadata[Array[Byte], Array[Byte]] = null
+    var messages = ArrayBuffer[String]()
+
+    while (message == null || message.offset < maxOffsetInclusive) {
+      message = stream.next
+      messages += new String(message.message, "UTF-8")
+      System.err.println("TestStatefulTask.readAll(): offset=%s, message=%s" format (message.offset, messages.last))
+    }
+
+    consumerConnector.shutdown
+
+    messages.toList
+  }
+}
+
+object TestTask {
+  val tasks = new HashMap[Partition, TestTask] with SynchronizedMap[Partition, TestTask]
+  @volatile var allTasksRegistered = new CountDownLatch(TestStatefulTask.TOTAL_PARTITIONS)
+
+  /**
+   * Static method that tasks can use to register themselves with. Useful so
+   * we don't have to sneak into the ThreadJob/SamzaContainer to get our test
+   * tasks.
+   */
+  def register(partition: Partition, task: TestTask) {
+    tasks += partition -> task
+    allTasksRegistered.countDown
+  }
+
+  def awaitTaskRegistered {
+    allTasksRegistered.await(60, TimeUnit.SECONDS)
+    assertEquals(0, allTasksRegistered.getCount)
+    assertEquals(TestStatefulTask.TOTAL_PARTITIONS, tasks.size)
+    // Reset the registered latch, so we can use it again every time we start a new job.
+    TestTask.allTasksRegistered = new CountDownLatch(TestStatefulTask.TOTAL_PARTITIONS)
+  }
+}
+
+class TestTask extends StreamTask with InitableTask {
+  var store: KeyValueStore[String, String] = null
+  var restored = Set[String]()
+  var received = ArrayBuffer[String]()
+  val initFinished = new CountDownLatch(1)
+  var gotMessage = new CountDownLatch(1)
+
+  def init(config: Config, context: TaskContext) {
+    TestTask.register(context.getPartition, this)
+    store = context
+      .getStore(TestStatefulTask.STATE_TOPIC)
+      .asInstanceOf[KeyValueStore[String, String]]
+    val iter = store.all
+    restored ++= iter
+      .map(_.getValue)
+      .toSet
+    System.err.println("TestTask.init(): %s" format restored)
+    iter.close
+    initFinished.countDown()
+  }
+
+  def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
+    val msg = envelope.getMessage.asInstanceOf[String]
+
+    System.err.println("TestTask.process(): %s" format msg)
+
+    received += msg
+    store.put(msg, msg)
+    coordinator.commit
+
+    // Notify sender that we got a message.
+    gotMessage.countDown
+  }
+
+  def awaitMessage {
+    gotMessage.await(60, TimeUnit.SECONDS)
+    assertEquals(0, gotMessage.getCount)
+    gotMessage = new CountDownLatch(1)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/resources/scalate/WEB-INF/layouts/default.scaml
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/resources/scalate/WEB-INF/layouts/default.scaml b/samza-yarn/src/main/resources/scalate/WEB-INF/layouts/default.scaml
new file mode 100644
index 0000000..6b4e43b
--- /dev/null
+++ b/samza-yarn/src/main/resources/scalate/WEB-INF/layouts/default.scaml
@@ -0,0 +1,27 @@
+-#
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you 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.
+-@ val title: String
+-@ val body: String
+
+!!!
+%html
+  %head
+    %title= title
+    %link(href="css/bootstrap.min.css" rel="stylesheet" media="screen")
+  %body
+    != body

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml
----------------------------------------------------------------------
diff --git a/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml b/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml
new file mode 100644
index 0000000..6530bad
--- /dev/null
+++ b/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml
@@ -0,0 +1,107 @@
+-#
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you 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.
+-@ val state: org.apache.samza.job.yarn.SamzaAppMasterState
+-@ val config: scala.collection.immutable.TreeMap[String, String]
+-@ val rmHttpAddress: String
+-@ val jobName: String = config.get("job.name").getOrElse("MISSING JOB NAME")
+-@ val username: String = org.apache.hadoop.security.UserGroupInformation.getCurrentUser.getShortUserName
+- attributes("title") = jobName
+
+%div.container
+  %h1= jobName
+  %table(class="table table-bordered" style="margin-top: 10px;")
+    %tr
+      %td(colspan="2")
+        %h4 Application Master
+    %tr
+      %td Hostname
+      %td
+        %a(target="_blank" href="http://#{state.nodeHost}:#{state.nodeHttpPort.toString}")= state.nodeHost
+    %tr
+      %td User
+      %td= username
+    %tr
+      %td Tracking port
+      %td= state.trackingPort.toString
+    %tr
+      %td RPC port
+      %td= state.rpcPort.toString
+    %tr
+      %td Attempt ID
+      %td= state.appAttemptId
+    %tr
+      %td Application master task ID
+      %td= state.taskId
+    %tr
+      %td Application master container
+      %td 
+        %a(target="_blank" href="http://#{state.nodeHost}:#{state.nodeHttpPort.toString}/node/containerlogs/#{state.containerId.toString}/#{username}")= state.containerId.toString
+    %tr
+      %td(colspan="2")
+        %h4 Containers
+    %tr
+      %td Running containers
+      %td
+        %ul
+          - for((taskId, container) <- state.runningTasks)
+            %li
+              Task group #{taskId.toString}
+              %ul
+                %li
+                  %a(target="_blank" href="http://#{container.getNodeHttpAddress}/node/containerlogs/#{container.getId.toString}/#{username}")= container.getId.toString
+                %li
+                  %a(target="_blank" href="http://#{container.getNodeHttpAddress}")= container.getNodeHttpAddress
+    %tr
+      %td Completed
+      %td= state.completedTasks.toString
+    %tr
+      %td Needed
+      %td= state.neededContainers.toString
+    %tr
+      %td Failed
+      %td= state.failedContainers.toString
+    %tr
+      %td Released
+      %td= state.releasedContainers.toString
+    %tr
+      %td(colspan="2")
+        %h4 Task Groups
+    %tr
+      %td Partition Assignment
+      %td
+        - for((taskId, partitions) <- state.taskPartitions)
+          = taskId
+          = "(Partitions: %s)" format partitions.map(_.getPartitionId).toList.sorted.mkString(", ")
+          %br
+    %tr
+      %td Total
+      %td= state.taskCount.toString
+    %tr
+      %td Unclaimed
+      %td= state.unclaimedTasks.size.toString
+    %tr
+      %td Finished
+      %td= state.finishedTasks.size.toString
+    %tr
+      %td(colspan="2")
+        %h4 Config
+    - for(entrySet <- new java.util.TreeMap[String, String](config.asInstanceOf[Map[String, String]]).entrySet)
+      %tr
+        %td= entrySet.getKey
+        %td= entrySet.getValue
+  %script(src="js/bootstrap.min.js")


[07/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala b/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala
new file mode 100644
index 0000000..164a2ee
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/JvmMetrics.scala
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics
+
+import java.lang.management.ManagementFactory
+import scala.collection._
+import scala.collection.JavaConversions._
+import java.lang.Thread.State._
+import java.util.concurrent.Executors
+import java.util.concurrent.TimeUnit
+import grizzled.slf4j.Logging
+import org.apache.samza.util.Util
+import org.apache.samza.util.DaemonThreadFactory
+
+/**
+ * Straight up ripoff of Hadoop's metrics2 JvmMetrics class.
+ */
+class JvmMetrics(group: String, registry: MetricsRegistry) extends Runnable with Logging {
+  final val M = 1024 * 1024.0f
+
+  def this(registry: MetricsRegistry) = this("samza.jvm", registry)
+
+  val memoryMXBean = ManagementFactory.getMemoryMXBean()
+  val gcBeans = ManagementFactory.getGarbageCollectorMXBeans()
+  val threadMXBean = ManagementFactory.getThreadMXBean()
+  var gcBeanCounters = Map[String, (Counter, Counter)]()
+  val executor = Executors.newScheduledThreadPool(1, new DaemonThreadFactory)
+
+  // jvm metrics
+  val gMemNonHeapUsedM = registry.newGauge[Float](group, "MemNonHeapUsedM", 0)
+  val gMemNonHeapCommittedM = registry.newGauge[Float](group, "MemNonHeapCommittedM", 0)
+  val gMemHeapUsedM = registry.newGauge[Float](group, "MemHeapUsedM", 0)
+  val gMemHeapCommittedM = registry.newGauge[Float](group, "MemHeapCommittedM", 0)
+  val gThreadsNew = registry.newGauge[Long](group, "ThreadsNew", 0)
+  val gThreadsRunnable = registry.newGauge[Long](group, "ThreadsRunnable", 0)
+  val gThreadsBlocked = registry.newGauge[Long](group, "ThreadsBlocked", 0)
+  val gThreadsWaiting = registry.newGauge[Long](group, "ThreadsWaiting", 0)
+  val gThreadsTimedWaiting = registry.newGauge[Long](group, "ThreadsTimedWaiting", 0)
+  val gThreadsTerminated = registry.newGauge[Long](group, "ThreadsTerminated", 0)
+  val cGcCount = registry.newCounter(group, "GcCount")
+  val cGcTimeMillis = registry.newCounter(group, "GcTimeMillis")
+
+  def start {
+    executor.scheduleWithFixedDelay(this, 0, 5, TimeUnit.SECONDS)
+  }
+
+  def run {
+    debug("updating jvm metrics")
+
+    updateMemoryUsage
+    updateGcUsage
+    updateThreadUsage
+
+    debug("updated metrics to: [%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s]" format
+      (gMemNonHeapUsedM, gMemNonHeapCommittedM, gMemHeapUsedM, gMemHeapCommittedM, gThreadsNew,
+        gThreadsRunnable, gThreadsBlocked, gThreadsWaiting, gThreadsTimedWaiting,
+        gThreadsTerminated, cGcCount, cGcTimeMillis))
+  }
+
+  def stop = executor.shutdown
+
+  private def updateMemoryUsage {
+    val memNonHeap = memoryMXBean.getNonHeapMemoryUsage()
+    val memHeap = memoryMXBean.getHeapMemoryUsage()
+    gMemNonHeapUsedM.set(memNonHeap.getUsed() / M)
+    gMemNonHeapCommittedM.set(memNonHeap.getCommitted() / M)
+    gMemHeapUsedM.set(memHeap.getUsed() / M)
+    gMemHeapCommittedM.set(memHeap.getCommitted() / M)
+  }
+
+  private def updateGcUsage {
+    var count = 0l
+    var timeMillis = 0l
+
+    gcBeans.foreach(gcBean => {
+      val c = gcBean.getCollectionCount()
+      val t = gcBean.getCollectionTime()
+      val gcInfo = getGcInfo(gcBean.getName)
+      gcInfo._1.inc(c - gcInfo._1.getCount())
+      gcInfo._2.inc(t - gcInfo._2.getCount())
+      count += c
+      timeMillis += t
+    })
+
+    cGcCount.inc(count - cGcCount.getCount())
+    cGcTimeMillis.inc(timeMillis - cGcTimeMillis.getCount())
+  }
+
+  private def getGcInfo(gcName: String): (Counter, Counter) = {
+    gcBeanCounters.get(gcName) match {
+      case Some(gcBeanCounterTuple) => gcBeanCounterTuple
+      case _ => {
+        val t = (registry.newCounter(group, "GcCount" + gcName), registry.newCounter(group, "GcTimeMillis" + gcName))
+        gcBeanCounters += (gcName -> t)
+        t
+      }
+    }
+  }
+
+  private def updateThreadUsage {
+    var threadsNew = 0l
+    var threadsRunnable = 0l
+    var threadsBlocked = 0l
+    var threadsWaiting = 0l
+    var threadsTimedWaiting = 0l
+    var threadsTerminated = 0l
+    var threadIds = threadMXBean.getAllThreadIds
+
+    threadMXBean.getThreadInfo(threadIds, 0).foreach(threadInfo =>
+      Option(threadInfo) match {
+        case Some(threadInfo) => {
+          threadInfo.getThreadState match {
+            case NEW => threadsNew += 1
+            case RUNNABLE => threadsRunnable += 1
+            case BLOCKED => threadsBlocked += 1
+            case WAITING => threadsWaiting += 1
+            case TIMED_WAITING => threadsTimedWaiting += 1
+            case TERMINATED => threadsTerminated += 1
+          }
+        }
+        case _ => // race protection
+      })
+
+    gThreadsNew.set(threadsNew)
+    gThreadsRunnable.set(threadsRunnable)
+    gThreadsBlocked.set(threadsBlocked)
+    gThreadsWaiting.set(threadsWaiting)
+    gThreadsTimedWaiting.set(threadsTimedWaiting)
+    gThreadsTerminated.set(threadsTerminated)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala
new file mode 100644
index 0000000..fc0bd38
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics
+import grizzled.slf4j.Logging
+import java.util.concurrent.ConcurrentHashMap
+
+/**
+ * A class that holds all metrics registered with it. It can be registered
+ * with one or more MetricReporters to flush metrics.
+ */
+class MetricsRegistryMap extends ReadableMetricsRegistry with Logging {
+  var listeners = Set[ReadableMetricsRegistryListener]()
+
+  /*
+   * groupName -> metricName -> metric
+   */
+  val metrics = new ConcurrentHashMap[String, ConcurrentHashMap[String, Metric]]
+
+  def newCounter(group: String, name: String) = {
+    debug("Creating new counter %s %s." format (group, name))
+    putAndGetGroup(group).putIfAbsent(name, new Counter(name))
+    val counter = metrics.get(group).get(name).asInstanceOf[Counter]
+    listeners.foreach(_.onCounter(group, counter))
+    counter
+  }
+
+  def newGauge[T](group: String, name: String, value: T) = {
+    debug("Creating new gauge %s %s %s." format (group, name, value))
+    putAndGetGroup(group).putIfAbsent(name, new Gauge[T](name, value))
+    val gauge = metrics.get(group).get(name).asInstanceOf[Gauge[T]]
+    listeners.foreach(_.onGauge(group, gauge))
+    gauge
+  }
+
+  private def putAndGetGroup(group: String) = {
+    metrics.putIfAbsent(group, new ConcurrentHashMap[String, Metric])
+    metrics.get(group)
+  }
+
+  def getGroups = metrics.keySet()
+
+  def getGroup(group: String) = metrics.get(group)
+
+  override def toString() = metrics.toString
+
+  def listen(listener: ReadableMetricsRegistryListener) {
+    listeners += listener
+  }
+
+  def unlisten(listener: ReadableMetricsRegistryListener) {
+    listeners -= listener
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala
new file mode 100644
index 0000000..8814e68
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.lang.management.ManagementFactory
+import grizzled.slf4j.Logging
+import javax.management.MBeanServer
+import javax.management.ObjectName
+import org.apache.samza.config.Config
+import org.apache.samza.metrics.Counter
+import org.apache.samza.metrics.Gauge
+import org.apache.samza.metrics.MetricsReporter
+import org.apache.samza.metrics.MetricsReporterFactory
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.metrics.ReadableMetricsRegistryListener
+import scala.collection.JavaConversions._
+import org.apache.samza.metrics.MetricsVisitor
+
+class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging {
+  var sources = Map[ReadableMetricsRegistry, String]()
+  var listeners = Map[ReadableMetricsRegistry, ReadableMetricsRegistryListener]()
+
+  def start() {
+    for ((registry, listener) <- listeners) {
+      // First, add a listener for all new metrics that are added.
+      registry.listen(listener)
+
+      // Second, add all existing metrics.
+      registry.getGroups.foreach(group => {
+        registry.getGroup(group).foreach {
+          case (name, metric) =>
+            metric.visit(new MetricsVisitor {
+              def counter(counter: Counter) = registerBean(new JmxCounter(counter, getObjectName(group, name, sources(registry))));
+              def gauge[T](gauge: Gauge[T]) = registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, name, sources(registry))))
+            })
+        }
+      })
+    }
+  }
+
+  def register(source: String, registry: ReadableMetricsRegistry) {
+    if (!listeners.contains(registry)) {
+      sources += registry -> source
+      listeners += registry -> new ReadableMetricsRegistryListener {
+        def onCounter(group: String, counter: Counter) {
+          registerBean(new JmxCounter(counter, getObjectName(group, counter.getName, source)))
+        }
+
+        def onGauge(group: String, gauge: Gauge[_]) {
+          registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, gauge.getName, source)))
+        }
+      }
+    } else {
+      warn("Trying to re-register a registry for source %s. Ignoring." format source)
+    }
+  }
+
+  def stop() {
+    for ((registry, listener) <- listeners) {
+      registry.unlisten(listener)
+    }
+  }
+
+  def getObjectName(group: String, name: String, t: String) = {
+    val nameBuilder = new StringBuilder
+    nameBuilder.append(makeNameJmxSafe(group))
+    nameBuilder.append(":type=")
+    nameBuilder.append(makeNameJmxSafe(t))
+    nameBuilder.append(",name=")
+    nameBuilder.append(makeNameJmxSafe(name))
+    val objName = new ObjectName(nameBuilder.toString)
+    debug("Resolved name for %s, %s, %s to: %s" format (group, name, t, objName))
+    objName
+  }
+
+  /*
+   * JMX only has ObjectName.quote, which is pretty nasty looking. This 
+   * function escapes without quoting, using the rules outlined in: 
+   * http://docs.oracle.com/javase/1.5.0/docs/api/javax/management/ObjectName.html
+   */
+  def makeNameJmxSafe(str: String) = str
+    .replace(",", "_")
+    .replace("=", "_")
+    .replace(":", "_")
+    .replace("\"", "_")
+    .replace("*", "_")
+    .replace("?", "_")
+
+  def registerBean(bean: MetricMBean) {
+    if (!server.isRegistered(bean.objectName)) {
+      debug("Registering MBean for %s." format bean.objectName)
+      server.registerMBean(bean, bean.objectName);
+    }
+  }
+}
+
+trait MetricMBean {
+  def objectName(): ObjectName
+}
+
+abstract class AbstractBean(val on: ObjectName) extends MetricMBean {
+  override def objectName = on
+}
+
+trait JmxGaugeMBean extends MetricMBean {
+  def getValue(): Object
+}
+
+class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extends JmxGaugeMBean {
+  def getValue = g.getValue
+  def objectName = on
+}
+
+trait JmxCounterMBean extends MetricMBean {
+  def getCount(): Long
+}
+
+class JmxCounter(c: org.apache.samza.metrics.Counter, on: ObjectName) extends JmxCounterMBean {
+  def getCount() = c.getCount()
+  def objectName = on
+}
+
+class JmxReporterFactory extends MetricsReporterFactory with Logging {
+  def getMetricsReporter(name: String, containerName: String, config: Config) = {
+    info("Creating JMX reporter with  name %s." format name)
+    new JmxReporter(ManagementFactory.getPlatformMBeanServer)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala
new file mode 100644
index 0000000..d7aec8b
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.util.Collections
+import java.util.HashMap
+import java.util.Map
+import scala.collection.JavaConversions._
+
+object Metrics {
+  def fromMap(map: Map[String, Map[String, Object]]): Metrics = {
+    new Metrics(map)
+  }
+}
+
+/**
+ * Immutable metrics snapshot.
+ */
+class Metrics(metrics: Map[String, Map[String, Object]]) {
+  val immutableMetrics = new HashMap[String, Map[String, Object]]
+
+  for (groupEntry <- metrics.entrySet) {
+    val immutableMetricGroup = new HashMap[String, Object]
+
+    for (metricEntry <- groupEntry.getValue.asInstanceOf[Map[String, Object]].entrySet) {
+      immutableMetricGroup.put(metricEntry.getKey, metricEntry.getValue)
+    }
+
+    immutableMetrics.put(groupEntry.getKey, Collections.unmodifiableMap(immutableMetricGroup))
+  }
+
+  def get[T](group: String, metricName: String) =
+    immutableMetrics.get(group).get(metricName).asInstanceOf[T]
+
+  def get(group: String) = immutableMetrics.get(group)
+
+  def getAsMap(): Map[String, Map[String, Object]] = Collections.unmodifiableMap(immutableMetrics)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsHeader.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsHeader.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsHeader.scala
new file mode 100644
index 0000000..369c718
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsHeader.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.util.HashMap
+import java.util.Map
+import scala.reflect.BeanProperty
+
+object MetricsHeader {
+  def fromMap(map: Map[String, Object]): MetricsHeader = {
+    new MetricsHeader(
+      map.get("job-name").toString,
+      map.get("job-id").toString,
+      map.get("container-name").toString,
+      map.get("source").toString,
+      map.get("version").toString,
+      map.get("samza-version").toString,
+      map.get("host").toString,
+      map.get("time").asInstanceOf[Number].longValue,
+      map.get("reset-time").asInstanceOf[Number].longValue)
+  }
+}
+
+/**
+ * Immutable metric header snapshot.
+ */
+class MetricsHeader(
+  @BeanProperty val jobName: String,
+  @BeanProperty val jobId: String,
+  @BeanProperty val containerName: String,
+  @BeanProperty val source: String,
+  @BeanProperty val version: String,
+  @BeanProperty val samzaVersion: String,
+  @BeanProperty val host: String,
+  @BeanProperty val time: Long,
+  @BeanProperty val resetTime: Long) {
+
+  def getAsMap: Map[String, Object] = {
+    val map = new HashMap[String, Object]
+    map.put("job-name", jobName)
+    map.put("job-id", jobId)
+    map.put("container-name", containerName)
+    map.put("source", source)
+    map.put("version", version)
+    map.put("samza-version", samzaVersion)
+    map.put("host", host)
+    map.put("time", time: java.lang.Long)
+    map.put("reset-time", resetTime: java.lang.Long)
+    map
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshot.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshot.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshot.scala
new file mode 100644
index 0000000..da775f7
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshot.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.util.Map
+import java.util.HashMap
+import scala.reflect.BeanProperty
+
+object MetricsSnapshot {
+  def fromMap(map: Map[String, Map[String, Object]]) = {
+    val header = MetricsHeader.fromMap(map.get("header"))
+    val metrics = Metrics.fromMap(map.get("metrics").asInstanceOf[Map[String, Map[String, Object]]])
+    new MetricsSnapshot(header, metrics)
+  }
+}
+
+class MetricsSnapshot(@BeanProperty val header: MetricsHeader, @BeanProperty val metrics: Metrics) {
+  def getAsMap(): Map[String, Object] = {
+    val map = new HashMap[String, Object]
+
+    map.put("header", header.getAsMap)
+    map.put("metrics", metrics.getAsMap)
+
+    map
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala
new file mode 100644
index 0000000..79c647e
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.util.HashMap
+import java.util.Map
+import scala.collection.JavaConversions._
+import grizzled.slf4j.Logging
+import org.apache.samza.metrics.Counter
+import org.apache.samza.metrics.Gauge
+import org.apache.samza.metrics.MetricsReporter
+import org.apache.samza.metrics.MetricsVisitor
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import java.util.concurrent.Executors
+import org.apache.samza.util.DaemonThreadFactory
+import java.util.concurrent.TimeUnit
+import org.apache.samza.serializers.Serializer
+import org.apache.samza.system.SystemProducer
+import org.apache.samza.system.SystemStream
+import org.apache.samza.system.OutgoingMessageEnvelope
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ */
+class MetricsSnapshotReporter(
+  producer: SystemProducer,
+  out: SystemStream,
+  jobName: String,
+  jobId: String,
+  containerName: String,
+  version: String,
+  samzaVersion: String,
+  host: String,
+  serializer: Serializer[MetricsSnapshot] = null,
+  clock: () => Long = () => { System.currentTimeMillis }) extends MetricsReporter with Runnable with Logging {
+
+  val executor = Executors.newScheduledThreadPool(1, new DaemonThreadFactory)
+  val resetTime = clock()
+  var registries = List[(String, ReadableMetricsRegistry)]()
+
+  info("got metrics snapshot reporter properties [job name: %s, job id: %s, containerName: %s, version: %s, samzaVersion: %s, host: %s]"
+    format (jobName, jobId, containerName, version, samzaVersion, host))
+
+  def start {
+    info("Starting producer.")
+
+    producer.start
+
+    info("Starting reporter timer.")
+
+    // TODO could make this configurable.
+    executor.scheduleWithFixedDelay(this, 0, 60, TimeUnit.SECONDS)
+  }
+
+  def register(source: String, registry: ReadableMetricsRegistry) {
+    registries ::= (source, registry)
+
+    info("Registering %s with producer." format source)
+
+    producer.register(source)
+  }
+
+  def stop = {
+    info("Stopping producer.")
+
+    producer.stop
+
+    info("Stopping reporter timer.")
+
+    executor.shutdown
+    executor.awaitTermination(60, TimeUnit.SECONDS)
+
+    if (!executor.isTerminated) {
+      warn("Unable to shutdown reporter timer.")
+    }
+  }
+
+  def run {
+    debug("Begin flushing metrics.")
+
+    for ((source, registry) <- registries) {
+      debug("Flushing metrics for %s." format source)
+
+      val metricsMsg = new HashMap[String, Map[String, Object]]
+
+      // metrics
+      registry.getGroups.foreach(group => {
+        val groupMsg = new HashMap[String, Object]
+
+        registry.getGroup(group).foreach {
+          case (name, metric) =>
+            metric.visit(new MetricsVisitor {
+              def counter(counter: Counter) = groupMsg.put(name, counter.getCount: java.lang.Long)
+              def gauge[T](gauge: Gauge[T]) = groupMsg.put(name, gauge.getValue.asInstanceOf[Object])
+            })
+        }
+
+        metricsMsg.put(group, groupMsg)
+      })
+
+      val header = new MetricsHeader(jobName, jobId, containerName, source, version, samzaVersion, host, clock(), resetTime)
+      val metrics = new Metrics(metricsMsg)
+
+      debug("Flushing metrics for %s to %s with header and map: header=%s, map=%s." format (source, out, header.getAsMap, metrics.getAsMap))
+
+      val metricsSnapshot = new MetricsSnapshot(header, metrics)
+      val maybeSerialized = if (serializer != null) {
+        serializer.toBytes(metricsSnapshot)
+      } else {
+        metricsSnapshot
+      }
+
+      producer.send(source, new OutgoingMessageEnvelope(out, host, null, maybeSerialized))
+
+      // Always commit, since we don't want metrics to get batched up.
+      producer.commit(source)
+    }
+
+    debug("Finished flushing metrics.")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala
new file mode 100644
index 0000000..f20dc36
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import java.net.InetAddress
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+import org.apache.samza.config.Config
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.config.MetricsConfig.Config2Metrics
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.SerializerConfig.Config2Serializer
+import org.apache.samza.config.TaskConfig.Config2Task
+import org.apache.samza.metrics.MetricsReporter
+import org.apache.samza.metrics.MetricsReporterFactory
+import org.apache.samza.util.Util
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.serializers.Serializer
+import org.apache.samza.serializers.SerdeFactory
+import org.apache.samza.system.SystemFactory
+
+class MetricsSnapshotReporterFactory extends MetricsReporterFactory with Logging {
+  def getMetricsReporter(name: String, containerName: String, config: Config): MetricsReporter = {
+    info("Creating new metrics snapshot reporter.")
+
+    val jobName = config
+      .getName
+      .getOrElse(throw new SamzaException("Job name must be defined in config."))
+
+    val jobId = config
+      .getJobId
+      .getOrElse(1.toString)
+
+    val taskClass = config
+      .getTaskClass
+      .getOrElse(throw new SamzaException("No task class defined for config."))
+
+    val version = Option(Class.forName(taskClass).getPackage.getImplementationVersion)
+      .getOrElse({
+        warn("Unable to find implementation version in jar's meta info. Defaulting to 0.0.1.")
+        "0.0.1"
+      })
+
+    val samzaVersion = Option(classOf[MetricsSnapshotReporterFactory].getPackage.getImplementationVersion)
+      .getOrElse({
+        warn("Unable to find implementation samza version in jar's meta info. Defaulting to 0.0.1.")
+        "0.0.1"
+      })
+
+    val metricsSystemStreamName = config
+      .getMetricsReporterStream(name)
+      .getOrElse(throw new SamzaException("No metrics stream defined in config."))
+
+    val systemStream = Util.getSystemStreamFromNames(metricsSystemStreamName)
+
+    info("Got system stream %s." format systemStream)
+
+    val systemName = systemStream.getSystem
+
+    val systemFactoryClassName = config
+      .getSystemFactory(systemName)
+      .getOrElse(throw new SamzaException("Trying to fetch system factory for system %s, which isn't defined in config." format systemName))
+
+    val systemFactory = Util.getObj[SystemFactory](systemFactoryClassName)
+
+    info("Got system factory %s." format systemFactory)
+
+    val registry = new MetricsRegistryMap
+
+    val producer = systemFactory.getProducer(systemName, config, registry)
+
+    info("Got producer %s." format producer)
+
+    val streamSerdeName = config.getStreamMsgSerde(systemStream)
+    val systemSerdeName = config.getSystemMsgSerde(systemName)
+    val serdeName = streamSerdeName.getOrElse(systemSerdeName.getOrElse(null))
+    val serde = if (serdeName != null) {
+      config.getSerdeClass(serdeName) match {
+        case Some(serdeName) =>
+          Util
+            .getObj[SerdeFactory[MetricsSnapshot]](serdeName)
+            .getSerde(serdeName, config)
+        case _ => null
+      }
+    } else {
+      null
+    }
+
+    info("Got serde %s." format serde)
+
+    val reporter = new MetricsSnapshotReporter(
+      producer,
+      systemStream,
+      jobName,
+      jobId,
+      containerName,
+      version,
+      samzaVersion,
+      InetAddress.getLocalHost().getHostName(),
+      serde)
+
+    reporter.register(this.getClass.getSimpleName.toString, registry)
+
+    reporter
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala b/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala
new file mode 100644
index 0000000..574c584
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+
+import scala.collection.JavaConversions._
+import org.codehaus.jackson.map.ObjectMapper
+import org.apache.samza.system.SystemStream
+import org.apache.samza.checkpoint.Checkpoint
+
+class CheckpointSerde extends Serde[Checkpoint] {
+  val jsonMapper = new ObjectMapper()
+
+  def fromBytes(bytes: Array[Byte]): Checkpoint = {
+    try {
+      val checkpointMap = jsonMapper
+        .readValue(bytes, classOf[java.util.Map[String, java.util.Map[String, String]]])
+        .flatMap {
+          case (systemName, streamToOffsetMap) =>
+            streamToOffsetMap.map { case (streamName, offset) => (new SystemStream(systemName, streamName), offset) }
+        }
+      return new Checkpoint(checkpointMap)
+    } catch {
+      case _ => return null
+    }
+  }
+
+  def toBytes(checkpoint: Checkpoint) = {
+    val offsetMap = asJavaMap(checkpoint
+      .getOffsets
+      // Convert Map[SystemStream, String] offset map to a iterable of tuples (system, stream, offset)
+      .map { case (systemStream, offset) => (systemStream.getSystem, systemStream.getStream, offset) }
+      // Group into a Map[String, (String, String, String)] by system
+      .groupBy(_._1)
+      // Group the tuples for each system into a Map[String, String] for stream to offsets
+      .map {
+        case (systemName, tuples) =>
+          val streamToOffestMap = asJavaMap(tuples
+            // Group the tuples by stream name
+            .groupBy(_._2)
+            // There should only ever be one SystemStream to offset mapping, so just 
+            // grab the first element from the tuple list for each stream.
+            .map { case (streamName, tuples) => (streamName, tuples.head._3) }
+            .toMap)
+          (systemName, streamToOffestMap)
+      }.toMap)
+
+    jsonMapper.writeValueAsBytes(offsetMap)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala b/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
new file mode 100644
index 0000000..4f3ff6e
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/serializers/SerdeManager.scala
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+
+import org.apache.samza.SamzaException
+import org.apache.samza.config.SerializerConfig
+import org.apache.samza.system.SystemStream
+import org.apache.samza.system.OutgoingMessageEnvelope
+import org.apache.samza.system.IncomingMessageEnvelope
+
+class SerdeManager(
+  serdes: Map[String, Serde[Object]] = Map(),
+  systemKeySerdes: Map[String, Serde[Object]] = Map(),
+  systemMessageSerdes: Map[String, Serde[Object]] = Map(),
+  systemStreamKeySerdes: Map[SystemStream, Serde[Object]] = Map(),
+  systemStreamMessageSerdes: Map[SystemStream, Serde[Object]] = Map(),
+  changeLogSystemStreams: Set[SystemStream] = Set()) {
+
+  def toBytes(obj: Object, serializerName: String) = serdes
+    .getOrElse(serializerName, throw new SamzaException("No serde defined for %s" format serializerName))
+    .toBytes(obj)
+
+  def toBytes(envelope: OutgoingMessageEnvelope): OutgoingMessageEnvelope = {
+    val key = if (changeLogSystemStreams.contains(envelope.getSystemStream)) {
+      // If the stream is a change log stream, don't do any serde. It is up to storage engines to handle serde.
+      envelope.getKey
+    } else if (envelope.getKeySerializerName != null) {
+      // If a serde is defined, use it.
+      toBytes(envelope.getKey, envelope.getKeySerializerName)
+    } else if (systemStreamKeySerdes.contains(envelope.getSystemStream)) {
+      // If the stream has a serde defined, use it.
+      systemStreamKeySerdes(envelope.getSystemStream).toBytes(envelope.getKey)
+    } else if (systemKeySerdes.contains(envelope.getSystemStream.getSystem)) {
+      // If the system has a serde defined, use it.
+      systemKeySerdes(envelope.getSystemStream.getSystem).toBytes(envelope.getKey)
+    } else {
+      // Just use the object.
+      envelope.getKey
+    }
+
+    val message = if (changeLogSystemStreams.contains(envelope.getSystemStream)) {
+      // If the stream is a change log stream, don't do any serde. It is up to storage engines to handle serde.
+      envelope.getMessage
+    } else if (envelope.getMessageSerializerName != null) {
+      // If a serde is defined, use it.
+      toBytes(envelope.getMessage, envelope.getMessageSerializerName)
+    } else if (systemStreamMessageSerdes.contains(envelope.getSystemStream)) {
+      // If the stream has a serde defined, use it.
+      systemStreamMessageSerdes(envelope.getSystemStream).toBytes(envelope.getMessage)
+    } else if (systemMessageSerdes.contains(envelope.getSystemStream.getSystem)) {
+      // If the system has a serde defined, use it.
+      systemMessageSerdes(envelope.getSystemStream.getSystem).toBytes(envelope.getMessage)
+    } else {
+      // Just use the object.
+      envelope.getMessage
+    }
+
+    new OutgoingMessageEnvelope(
+      envelope.getSystemStream,
+      null,
+      null,
+      envelope.getPartitionKey,
+      key,
+      message)
+  }
+
+  def fromBytes(bytes: Array[Byte], deserializerName: String) = serdes
+    .getOrElse(deserializerName, throw new SamzaException("No serde defined for %s" format deserializerName))
+    .fromBytes(bytes)
+
+  def fromBytes(envelope: IncomingMessageEnvelope) = {
+    val key = if (changeLogSystemStreams.contains(envelope.getSystemStreamPartition.getSystemStream)) {
+      // If the stream is a change log stream, don't do any serde. It is up to storage engines to handle serde.
+      envelope.getKey
+    } else if (systemStreamKeySerdes.contains(envelope.getSystemStreamPartition)) {
+      // If the stream has a serde defined, use it.
+      systemStreamKeySerdes(envelope.getSystemStreamPartition).fromBytes(envelope.getKey.asInstanceOf[Array[Byte]])
+    } else if (systemKeySerdes.contains(envelope.getSystemStreamPartition.getSystem)) {
+      // If the system has a serde defined, use it.
+      systemKeySerdes(envelope.getSystemStreamPartition.getSystem).fromBytes(envelope.getKey.asInstanceOf[Array[Byte]])
+    } else {
+      // Just use the object.
+      envelope.getKey
+    }
+
+    val message = if (changeLogSystemStreams.contains(envelope.getSystemStreamPartition.getSystemStream)) {
+      // If the stream is a change log stream, don't do any serde. It is up to storage engines to handle serde.
+      envelope.getMessage
+    } else if (systemStreamMessageSerdes.contains(envelope.getSystemStreamPartition)) {
+      // If the stream has a serde defined, use it.
+      systemStreamMessageSerdes(envelope.getSystemStreamPartition).fromBytes(envelope.getMessage.asInstanceOf[Array[Byte]])
+    } else if (systemMessageSerdes.contains(envelope.getSystemStreamPartition.getSystem)) {
+      // If the system has a serde defined, use it.
+      systemMessageSerdes(envelope.getSystemStreamPartition.getSystem).fromBytes(envelope.getMessage.asInstanceOf[Array[Byte]])
+    } else {
+      // Just use the object.
+      envelope.getMessage
+    }
+
+    new IncomingMessageEnvelope(
+      envelope.getSystemStreamPartition,
+      envelope.getOffset,
+      key,
+      message)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/serializers/StringSerde.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/StringSerde.scala b/samza-core/src/main/scala/org/apache/samza/serializers/StringSerde.scala
new file mode 100644
index 0000000..45cde95
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/serializers/StringSerde.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers
+
+import java.nio.ByteBuffer
+import org.apache.samza.config.Config
+
+/**
+ * A serializer for strings
+ */
+class StringSerdeFactory extends SerdeFactory[String] {
+  def getSerde(name: String, config: Config): Serde[String] =
+    new StringSerde(config.get("encoding", "UTF-8"))
+}
+
+class StringSerde(val encoding: String) extends Serde[String] {
+  def toBytes(obj: String): Array[Byte] =
+    obj.toString.getBytes(encoding)
+
+  def fromBytes(bytes: Array[Byte]): String =
+    new String(bytes, 0, bytes.size, encoding)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
new file mode 100644
index 0000000..f4c0194
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage
+
+import java.io.File
+import scala.collection.Map
+import grizzled.slf4j.Logging
+import org.apache.samza.Partition
+import org.apache.samza.system.SystemConsumer
+import org.apache.samza.system.SystemStream
+import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.system.SystemStreamPartitionIterator
+import org.apache.samza.task.MessageCollector
+import org.apache.samza.util.Util
+
+object TaskStorageManager {
+  def getStoreDir(storeBaseDir: File, storeName: String) = {
+    new File(storeBaseDir, storeName)
+  }
+
+  def getStorePartitionDir(storeBaseDir: File, storeName: String, partition: Partition) = {
+    new File(storeBaseDir, storeName + File.separator + partition.getPartitionId)
+  }
+}
+
+/**
+ * Manage all the storage engines for a given task
+ */
+class TaskStorageManager(
+  partition: Partition,
+  taskStores: Map[String, StorageEngine] = Map(),
+  storeConsumers: Map[String, SystemConsumer] = Map(),
+  changeLogSystemStreams: Map[String, SystemStream] = Map(),
+  storeBaseDir: File = new File(System.getProperty("user.dir"), "state")) extends Logging {
+
+  def apply(storageEngineName: String) = taskStores(storageEngineName)
+
+  def init(collector: MessageCollector) {
+    cleanBaseDirs
+    startConsumers
+    restoreStores(collector)
+    stopConsumers
+  }
+
+  private def cleanBaseDirs {
+    debug("Cleaning base directories for stores.")
+
+    taskStores.keys.foreach(storeName => {
+      val storagePartitionDir = TaskStorageManager.getStorePartitionDir(storeBaseDir, storeName, partition)
+
+      debug("Cleaning %s for store %s." format (storagePartitionDir, storeName))
+
+      Util.rm(storagePartitionDir)
+      storagePartitionDir.mkdirs
+    })
+  }
+
+  private def startConsumers {
+    debug("Starting consumers for stores.")
+
+    for ((storeName, systemStream) <- changeLogSystemStreams) {
+      val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
+      val consumer = storeConsumers(storeName)
+
+      debug("Registering consumer for system stream partition %s." format systemStreamPartition)
+
+      consumer.register(systemStreamPartition, null)
+    }
+
+    storeConsumers.values.foreach(_.start)
+  }
+
+  private def restoreStores(collector: MessageCollector) {
+    debug("Restoring stores.")
+
+    for ((storeName, store) <- taskStores) {
+      if (changeLogSystemStreams.contains(storeName)) {
+        val systemStream = changeLogSystemStreams(storeName)
+        val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
+        val systemConsumer = storeConsumers(storeName)
+        val systemConsumerIterator = new SystemStreamPartitionIterator(systemConsumer, systemStreamPartition);
+        store.restore(systemConsumerIterator)
+      }
+    }
+  }
+
+  private def stopConsumers {
+    debug("Stopping consumers for stores.")
+
+    storeConsumers.values.foreach(_.stop)
+  }
+
+  def flush() {
+    debug("Flushing stores.")
+
+    taskStores.values.foreach(_.flush)
+  }
+
+  def stop() {
+    debug("Stopping stores.")
+
+    taskStores.values.foreach(_.stop)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/system/DefaultPicker.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/system/DefaultPicker.scala b/samza-core/src/main/scala/org/apache/samza/system/DefaultPicker.scala
new file mode 100644
index 0000000..8bad75c
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/system/DefaultPicker.scala
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system
+
+import java.util.ArrayDeque
+
+class DefaultPicker extends IncomingMessageEnvelopePicker {
+  var q = new ArrayDeque[IncomingMessageEnvelope]()
+  def update(envelope: IncomingMessageEnvelope) = q.add(envelope)
+  def pick = q.poll
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala b/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala
new file mode 100644
index 0000000..2e6f3b8
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.Queue
+import grizzled.slf4j.Logging
+import org.apache.samza.serializers.SerdeManager
+
+class SystemConsumers(
+  picker: IncomingMessageEnvelopePicker,
+  consumers: Map[String, SystemConsumer],
+  serdeManager: SerdeManager,
+  maxMsgsPerStreamPartition: Int = 1000,
+  noNewMessagesTimeout: Long = 10) extends Logging {
+
+  // TODO add metrics
+
+  var unprocessedMessages = Map[SystemStreamPartition, Queue[IncomingMessageEnvelope]]()
+  var neededByPicker = Set[SystemStreamPartition]()
+  var fetchMap = Map[SystemStreamPartition, java.lang.Integer]()
+  var timeout = noNewMessagesTimeout
+
+  debug("Got stream consumers: %s" format consumers)
+  debug("Got max messages per stream: %s" format maxMsgsPerStreamPartition)
+  debug("Got no new message timeout: %s" format noNewMessagesTimeout)
+
+  def start {
+    debug("Starting consumers.")
+
+    consumers.values.foreach(_.start)
+  }
+
+  def stop {
+    debug("Stopping consumers.")
+
+    consumers.values.foreach(_.stop)
+  }
+
+  def register(systemStreamPartition: SystemStreamPartition, lastReadOffset: String) {
+    debug("Registering stream: %s, %s" format (systemStreamPartition, lastReadOffset))
+
+    neededByPicker += systemStreamPartition
+    fetchMap += systemStreamPartition -> maxMsgsPerStreamPartition
+    unprocessedMessages += systemStreamPartition -> Queue[IncomingMessageEnvelope]()
+    consumers(systemStreamPartition.getSystem).register(systemStreamPartition, lastReadOffset)
+  }
+
+  def pick = {
+    val picked = picker.pick
+
+    if (picked == null) {
+      debug("Picker returned null.")
+
+      // Allow blocking if the picker didn't pick a message.
+      timeout = noNewMessagesTimeout
+    } else {
+      debug("Picker returned an incoming message envelope: %s" format picked)
+
+      // Don't block if we have a message to process.
+      timeout = 0
+
+      // Ok to give the picker a new message from this stream.
+      neededByPicker += picked.getSystemStreamPartition
+    }
+
+    refresh
+    picked
+  }
+
+  private def refresh {
+    debug("Refreshing picker with new messages.")
+
+    // Poll every system for new messages.
+    consumers.keys.foreach(poll(_))
+
+    // Update the picker.
+    neededByPicker.foreach(systemStreamPartition =>
+      // If we have messages for a stream that the picker needs, then update.
+      if (fetchMap(systemStreamPartition).intValue < maxMsgsPerStreamPartition) {
+        picker.update(unprocessedMessages(systemStreamPartition).dequeue)
+        fetchMap += systemStreamPartition -> (fetchMap(systemStreamPartition).intValue + 1)
+        neededByPicker -= systemStreamPartition
+      })
+  }
+
+  private def poll(systemName: String) = {
+    debug("Polling system consumer: %s" format systemName)
+
+    val consumer = consumers(systemName)
+
+    debug("Filtering for system: %s, %s" format (systemName, fetchMap))
+
+    val systemFetchMap = fetchMap.filterKeys(_.getSystem.equals(systemName))
+
+    debug("Fetching: %s" format systemFetchMap)
+
+    val incomingEnvelopes = consumer.poll(systemFetchMap, timeout)
+
+    debug("Got incoming message envelopes: %s" format incomingEnvelopes)
+
+    // We have new un-processed envelopes, so update maps accordingly.
+    incomingEnvelopes.foreach(envelope => {
+      val systemStreamPartition = envelope.getSystemStreamPartition
+
+      debug("Got message for: %s, %s" format (systemStreamPartition, envelope))
+
+      fetchMap += systemStreamPartition -> (fetchMap(systemStreamPartition).intValue - 1)
+
+      debug("Updated fetch map for: %s, %s" format (systemStreamPartition, fetchMap))
+
+      unprocessedMessages(envelope.getSystemStreamPartition).enqueue(serdeManager.fromBytes(envelope))
+
+      debug("Updated unprocessed messages for: %s, %s" format (systemStreamPartition, unprocessedMessages))
+    })
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala b/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala
new file mode 100644
index 0000000..738d04f
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system
+
+import org.apache.samza.serializers.SerdeManager
+
+class SystemProducers(
+  producers: Map[String, SystemProducer],
+  serdeManager: SerdeManager) {
+
+  // TODO add metrics and logging
+
+  def start {
+    producers.values.foreach(_.start)
+  }
+
+  def stop {
+    producers.values.foreach(_.stop)
+  }
+
+  def register(source: String) {
+    producers.values.foreach(_.register(source))
+  }
+
+  def commit(source: String) {
+    producers.values.foreach(_.commit(source))
+  }
+
+  def send(source: String, envelope: OutgoingMessageEnvelope) {
+    producers(envelope.getSystemStream.getSystem).send(source, serdeManager.toBytes(envelope))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/task/ReadableCollector.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/task/ReadableCollector.scala b/samza-core/src/main/scala/org/apache/samza/task/ReadableCollector.scala
new file mode 100644
index 0000000..0ec79da
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/task/ReadableCollector.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task
+
+import scala.collection.mutable
+
+import org.apache.samza.system.OutgoingMessageEnvelope
+
+/** An in-memory implementation of MessageCollector that stores all outgoing messages in a list */
+class ReadableCollector extends MessageCollector {
+  var envelopes = new mutable.ArrayBuffer[OutgoingMessageEnvelope]()
+
+  def send(envelope: OutgoingMessageEnvelope) {
+    envelopes += envelope
+  }
+
+  def getEnvelopes = envelopes
+
+  def reset() = envelopes.clear
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/task/ReadableCoordinator.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/task/ReadableCoordinator.scala b/samza-core/src/main/scala/org/apache/samza/task/ReadableCoordinator.scala
new file mode 100644
index 0000000..aaf631e
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/task/ReadableCoordinator.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task
+
+/** An in-memory implementation of TaskCoordinator that stores all coordination messages */
+class ReadableCoordinator extends TaskCoordinator {
+  var commitRequested = false
+  var shutdownRequested = false
+
+  def commit { commitRequested = true }
+
+  def isCommitRequested = commitRequested
+
+  def shutdown { shutdownRequested = true }
+
+  def isShutdownRequested = shutdownRequested
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/util/DaemonThreadFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/util/DaemonThreadFactory.scala b/samza-core/src/main/scala/org/apache/samza/util/DaemonThreadFactory.scala
new file mode 100644
index 0000000..04e67a2
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/util/DaemonThreadFactory.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util
+
+import java.util.concurrent.ThreadFactory
+
+class DaemonThreadFactory extends ThreadFactory {
+  def newThread(r: Runnable) = {
+    val thread = new Thread(r)
+    thread.setDaemon(true)
+    thread
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/util/Util.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/util/Util.scala b/samza-core/src/main/scala/org/apache/samza/util/Util.scala
new file mode 100644
index 0000000..8386324
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/util/Util.scala
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util
+
+import java.io.File
+import java.net.InetAddress
+import java.net.UnknownHostException
+import java.util.Random
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+import org.apache.samza.config.Config
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.TaskConfig.Config2Task
+import scala.collection.JavaConversions._
+import java.util.concurrent.ThreadFactory
+import org.apache.samza.system.SystemFactory
+import org.apache.samza.system.SystemStream
+
+object Util extends Logging {
+  val random = new Random
+
+  /**
+   * Make an environment variable string safe to pass.
+   */
+  def envVarEscape(str: String) = str.replace("\"", "\\\"").replace("'", "\\'")
+
+  /**
+   * Get a random number >= startInclusive, and < endExclusive.
+   */
+  def randomBetween(startInclusive: Int, endExclusive: Int) =
+    startInclusive + random.nextInt(endExclusive - startInclusive)
+
+  /**
+   * Recursively remove a directory (or file), and all sub-directories. Equivalent
+   * to rm -rf.
+   */
+  def rm(file: File) {
+    if (file == null) {
+      return
+    } else if (file.isDirectory) {
+      val files = file.listFiles()
+      if (files != null) {
+        for (f <- files)
+          rm(f)
+      }
+      file.delete()
+    } else {
+      file.delete()
+    }
+  }
+
+  /**
+   * Instantiate a class instance from a given className.
+   */
+  def getObj[T](className: String) = {
+    Class
+      .forName(className)
+      .newInstance
+      .asInstanceOf[T]
+  }
+
+  /**
+   * Uses config to create SystemAdmin classes for all input stream systems to
+   * get each input stream's partition count, then returns the maximum count.
+   * An input stream with two partitions, and a second input stream with four
+   * partitions would result in this method returning 4.
+   */
+  def getMaxInputStreamPartitions(config: Config) = {
+    val inputStreams = config.getInputStreams
+    val systemNames = config.getSystemNames
+    val systemAdmins = systemNames.map(systemName => {
+      val systemFactoryClassName = config
+        .getSystemFactory(systemName)
+        .getOrElse(throw new SamzaException("A stream uses system %s, which is missing from the configuration." format systemName))
+      val systemFactory = Util.getObj[SystemFactory](systemFactoryClassName)
+      val systemAdmin = systemFactory.getAdmin(systemName, config)
+      (systemName, systemAdmin)
+    }).toMap
+    inputStreams.flatMap(systemStream => {
+      systemAdmins.get(systemStream.getSystem) match {
+        case Some(sysAdmin) => sysAdmin.getPartitions(systemStream.getStream)
+        case None => throw new IllegalArgumentException("Could not find a stream admin for system '" + systemStream.getSystem + "'")
+      }
+    }).toSet
+  }
+
+  /**
+   * Returns a SystemStream object based on the system stream name given. For 
+   * example, kafka.topic would return new SystemStream("kafka", "topic").
+   */
+  def getSystemStreamFromNames(systemStreamNames: String): SystemStream = {
+    val idx = systemStreamNames.indexOf('.')
+    if(idx < 0)
+      throw new IllegalArgumentException("No '.' in stream name '" + systemStreamNames + "'. Stream names should be in the form 'system.stream'")
+    new SystemStream(systemStreamNames.substring(0, idx), systemStreamNames.substring(idx + 1, systemStreamNames.length))
+  }
+  
+  /**
+   * Returns a SystemStream object based on the system stream name given. For 
+   * example, kafka.topic would return new SystemStream("kafka", "topic").
+   */
+  def getNameFromSystemStream(systemStream: SystemStream) = {
+    systemStream.getSystem + "." + systemStream.getStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/resources/test.properties
----------------------------------------------------------------------
diff --git a/samza-core/src/test/resources/test.properties b/samza-core/src/test/resources/test.properties
new file mode 100644
index 0000000..9348c7d
--- /dev/null
+++ b/samza-core/src/test/resources/test.properties
@@ -0,0 +1,24 @@
+#
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+#
+#
+
+job.factory.class=org.apache.samza.job.MockJobFactory
+job.name=test-job
+foo=bar

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala
new file mode 100644
index 0000000..50d9a05
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint.file
+
+import java.io.File
+import scala.collection.JavaConversions._
+import java.util.Random
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.SamzaException
+import org.apache.samza.Partition
+import org.apache.samza.checkpoint.Checkpoint
+import org.apache.samza.system.SystemStream
+
+class TestFileSystemCheckpointManager {
+  val checkpointRoot = System.getProperty("java.io.tmpdir")
+
+  @Test
+  def testReadForCheckpointFileThatDoesNotExistShouldReturnNull {
+    val cpm = new FileSystemCheckpointManager("some-job-name", new File(checkpointRoot))
+    assert(cpm.readLastCheckpoint(new Partition(1)) == null)
+  }
+
+  @Test
+  def testReadForCheckpointFileThatDoesExistShouldReturnProperCheckpoint {
+    val cpm = new FileSystemCheckpointManager("some-job-name", new File(checkpointRoot))
+    val partition = new Partition(2)
+    val cp = new Checkpoint(Map(
+      new SystemStream("a", "b") -> "c",
+      new SystemStream("a", "c") -> "d",
+      new SystemStream("b", "d") -> "e"))
+    cpm.start
+    cpm.writeCheckpoint(partition, cp)
+    val readCp = cpm.readLastCheckpoint(partition)
+    cpm.stop
+    assert(readCp.equals(cp))
+  }
+
+  @Test
+  def testMissingRootDirectoryShouldFailOnManagerCreation {
+    val cpm = new FileSystemCheckpointManager("some-job-name", new File(checkpointRoot + new Random().nextInt))
+    try {
+      cpm.start
+      fail("Expected an exception since root directory for fs checkpoint manager doesn't exist.")
+    } catch {
+      case e: SamzaException => None // this is expected
+    }
+    cpm.stop
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/config/factories/TestPropertiesConfigFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/config/factories/TestPropertiesConfigFactory.scala b/samza-core/src/test/scala/org/apache/samza/config/factories/TestPropertiesConfigFactory.scala
new file mode 100644
index 0000000..f254741
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/config/factories/TestPropertiesConfigFactory.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config.factories
+import java.net.URI
+import java.io.File
+import org.apache.samza.SamzaException
+import org.junit.Assert._
+import org.junit.Test
+
+class TestPropertiesConfigFactory {
+  val factory = new PropertiesConfigFactory()
+
+  @Test
+  def testCanReadPropertiesConfigFiles {
+    val config = factory.getConfig(URI.create("file://%s/src/test/resources/test.properties" format new File(".").getCanonicalPath))
+    assert("bar".equals(config.get("foo")))
+  }
+
+  @Test
+  def testCanNotReadNonLocalPropertiesConfigFiles {
+    try {
+      factory.getConfig(URI.create("hdfs://foo"))
+      fail("should have gotten a samza exception")
+    } catch {
+      case e: SamzaException => None // Do nothing
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/job/TestJobRunner.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/job/TestJobRunner.scala b/samza-core/src/test/scala/org/apache/samza/job/TestJobRunner.scala
new file mode 100644
index 0000000..21d8a78
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/job/TestJobRunner.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job
+import java.io.File
+import org.apache.samza.config.Config
+import org.junit.Assert._
+import org.junit.Test
+
+object TestJobRunner {
+  var processCount = 0
+}
+
+class TestJobRunner {
+  @Test
+  def testJobRunnerWorks {
+    JobRunner.main(Array(
+      "--config-factory",
+      "org.apache.samza.config.factories.PropertiesConfigFactory",
+      "--config-path",
+      "file://%s/src/test/resources/test.properties" format new File(".").getCanonicalPath))
+    assert(TestJobRunner.processCount == 1)
+  }
+}
+
+class MockJobFactory extends StreamJobFactory {
+  def getJob(config: Config): StreamJob = {
+    return new StreamJob {
+      def submit() = { TestJobRunner.processCount += 1; this }
+      def kill() = this
+      def waitForFinish(timeoutMs: Long) = ApplicationStatus.SuccessfulFinish
+      def waitForStatus(status: ApplicationStatus, timeoutMs: Long) = status
+      def getStatus() = ApplicationStatus.SuccessfulFinish
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/job/local/TestProcessJob.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/job/local/TestProcessJob.scala b/samza-core/src/test/scala/org/apache/samza/job/local/TestProcessJob.scala
new file mode 100644
index 0000000..d56024d
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/job/local/TestProcessJob.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.local;
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.job.ApplicationStatus
+
+class TestProcessJob {
+  @Test
+  def testProcessJobShouldFinishOnItsOwn {
+    val builder = new ProcessBuilder("sleep", "1")
+    val job = new ProcessJob(builder)
+    job.submit
+    job.waitForFinish(999999)
+  }
+
+  @Test
+  def testProcessJobKillShouldWork {
+    val builder = new ProcessBuilder("sleep", "999999999")
+    val job = new ProcessJob(builder)
+    job.submit
+    job.waitForFinish(500)
+    job.kill
+    job.waitForFinish(999999)
+    assert(ApplicationStatus.UnsuccessfulFinish.equals(job.waitForFinish(999999999)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/job/local/TestThreadJob.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/job/local/TestThreadJob.scala b/samza-core/src/test/scala/org/apache/samza/job/local/TestThreadJob.scala
new file mode 100644
index 0000000..7d45889
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/job/local/TestThreadJob.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.local
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.job.ApplicationStatus
+
+class TestThreadJob {
+  @Test
+  def testThreadJobShouldFinishOnItsOwn {
+    val job = new ThreadJob(new Runnable {
+      override def run {
+      }
+    })
+    job.submit
+    job.waitForFinish(999999)
+  }
+
+  @Test
+  def testThreadJobKillShouldWork {
+    val job = new ThreadJob(new Runnable {
+      override def run {
+        Thread.sleep(999999)
+      }
+    })
+    job.submit
+    job.waitForFinish(500)
+    job.kill
+    job.waitForFinish(999999)
+    assert(ApplicationStatus.UnsuccessfulFinish.equals(job.waitForFinish(999999999)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/metrics/TestJmxServer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/metrics/TestJmxServer.scala b/samza-core/src/test/scala/org/apache/samza/metrics/TestJmxServer.scala
new file mode 100644
index 0000000..f5594d0
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/metrics/TestJmxServer.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics
+
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.samza.config.MapConfig
+import grizzled.slf4j.Logging
+import javax.management.remote.{JMXConnector, JMXConnectorFactory, JMXServiceURL}
+import java.io.IOException
+
+
+class TestJmxServer extends Logging {
+  @Test
+  def serverStartsUp {
+    var jmxServer:JmxServer = null
+
+    try {
+      jmxServer = new JmxServer
+
+      println("Got jmxServer on port " + jmxServer.getPort)
+
+      val jmxURL = new JMXServiceURL(jmxServer.getJmxUrl)
+      var jmxConnector:JMXConnector = null
+      try {
+        jmxConnector = JMXConnectorFactory.connect(jmxURL, null)
+        val connection = jmxConnector.getMBeanServerConnection()
+        assertTrue("Connected but mbean count is somehow 0", connection.getMBeanCount.intValue() > 0)
+      } catch {
+        case ioe:IOException => fail("Couldn't open connection to local JMX server")
+      }finally {
+        if(jmxConnector != null) jmxConnector.close
+      }
+
+    } finally {
+      if (jmxServer != null) jmxServer.stop
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/metrics/reporter/TestJmxReporter.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/metrics/reporter/TestJmxReporter.scala b/samza-core/src/test/scala/org/apache/samza/metrics/reporter/TestJmxReporter.scala
new file mode 100644
index 0000000..8827697
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/metrics/reporter/TestJmxReporter.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics.reporter
+
+import org.junit.Assert._
+import org.junit.AfterClass
+import org.junit.BeforeClass
+import org.junit.Test
+import scala.collection.JavaConversions._
+import org.apache.samza.task.TaskContext
+import javax.management.remote.JMXConnectorFactory
+import org.apache.samza.metrics.MetricsRegistryMap
+import javax.management.remote.JMXConnectorServerFactory
+import javax.management.remote.JMXConnectorServer
+import java.rmi.registry.LocateRegistry
+import javax.management.remote.JMXServiceURL
+import org.apache.samza.config.MapConfig
+import java.lang.management.ManagementFactory
+import org.apache.samza.Partition
+import javax.management.ObjectName
+import org.apache.samza.metrics.JvmMetrics
+
+object TestJmxReporter {
+  val port = 4500
+  val url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://localhost:%d/jmxapitestrmi" format port)
+  var cs: JMXConnectorServer = null
+
+  @BeforeClass
+  def beforeSetupServers {
+    LocateRegistry.createRegistry(4500)
+    val mbs = ManagementFactory.getPlatformMBeanServer()
+    cs = JMXConnectorServerFactory.newJMXConnectorServer(url, null, mbs)
+    cs.start
+  }
+
+  @AfterClass
+  def afterCleanLogDirs {
+    if (cs != null) {
+      cs.stop
+    }
+  }
+}
+
+class TestJmxReporter {
+  import TestJmxReporter.url
+
+  @Test
+  def testJmxReporter {
+    val registry = new MetricsRegistryMap
+    val jvm = new JvmMetrics("test", registry)
+    val reporter = new JmxReporterFactory().getMetricsReporter("", "", new MapConfig(Map[String, String]()))
+
+    reporter.register("test", registry)
+    reporter.start
+    jvm.run
+
+    val mbserver = JMXConnectorFactory.connect(url).getMBeanServerConnection
+    val stateViaJMX = mbserver.getAttribute(new ObjectName("test:type=test,name=MemNonHeapUsedM"), "Value").asInstanceOf[Float]
+
+    assertTrue(stateViaJMX > 0)
+
+    reporter.stop
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/test/scala/org/apache/samza/task/TestReadableCoordinator.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/task/TestReadableCoordinator.scala b/samza-core/src/test/scala/org/apache/samza/task/TestReadableCoordinator.scala
new file mode 100644
index 0000000..c45ed9b
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/task/TestReadableCoordinator.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task
+
+import org.junit.Assert._
+import org.junit.Test
+
+class TestReadableCoordinator {
+  @Test
+  def testCommit {
+    val coord = new ReadableCoordinator
+    assert(!coord.isCommitRequested)
+    coord.commit
+    assert(coord.isCommitRequested)
+    coord.commit
+    assert(coord.isCommitRequested)
+  }
+
+  @Test
+  def testShutdown {
+    val coord = new ReadableCoordinator
+    assert(!coord.isShutdownRequested)
+    coord.shutdown
+    assert(coord.isShutdownRequested)
+    coord.shutdown
+    assert(coord.isShutdownRequested)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT-test.jar
----------------------------------------------------------------------
diff --git a/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT-test.jar b/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT-test.jar
new file mode 100644
index 0000000..ce88269
Binary files /dev/null and b/samza-kafka/lib/kafka_2.8.1-0.8.1-SNAPSHOT-test.jar differ


[08/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/FileSystemCheckpointManagerConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/FileSystemCheckpointManagerConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/FileSystemCheckpointManagerConfig.scala
new file mode 100644
index 0000000..707ea59
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/FileSystemCheckpointManagerConfig.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+object FileSystemCheckpointManagerConfig {
+  // file system checkpoint manager config constants
+  val CHECKPOINT_MANAGER_ROOT = "task.checkpoint.path" // system name to use when sending offset checkpoints
+
+  implicit def Config2FSCP(config: Config) = new FileSystemCheckpointManagerConfig(config)
+}
+
+class FileSystemCheckpointManagerConfig(config: Config) extends ScalaMapConfig(config) {
+  def getFileSystemCheckpointRoot = getOption(FileSystemCheckpointManagerConfig.CHECKPOINT_MANAGER_ROOT)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
new file mode 100644
index 0000000..150ea93
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+object JobConfig {
+  // job config constants
+  val STREAM_JOB_FACTORY_CLASS = "job.factory.class" // streaming.job_factory_class
+  val CONFIG_REWRITERS = "job.config.rewriters" // CSV list of config rewriter classes to apply
+  val CONFIG_REWRITER_CLASS = "job.config.rewriter.%s.class"
+  val JOB_NAME = "job.name" // streaming.job_name
+  val JOB_ID = "job.id" // streaming.job_id
+
+  implicit def Config2Job(config: Config) = new JobConfig(config)
+}
+
+class JobConfig(config: Config) extends ScalaMapConfig(config) {
+  def getName = getOption(JobConfig.JOB_NAME)
+
+  def getStreamJobFactoryClass = getOption(JobConfig.STREAM_JOB_FACTORY_CLASS)
+
+  def getJobId = getOption(JobConfig.JOB_ID)
+
+  def getConfigRewriters = getOption(JobConfig.CONFIG_REWRITERS)
+
+  def getConfigRewriterClass(name: String) = getOption(JobConfig.CONFIG_REWRITER_CLASS format name)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/MetricsConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/MetricsConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/MetricsConfig.scala
new file mode 100644
index 0000000..f7a11c5
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/MetricsConfig.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+import scala.collection.JavaConversions._
+
+object MetricsConfig {
+  // metrics config constants
+  val METRICS_REPORTERS = "metrics.reporters"
+  val METRICS_REPORTER_FACTORY = "metrics.reporter.%s.class"
+  val METRICS_SNAPSHOT_REPORTER_STREAM = "metrics.reporter.%s.stream"
+
+  implicit def Config2Metrics(config: Config) = new MetricsConfig(config)
+}
+
+class MetricsConfig(config: Config) extends ScalaMapConfig(config) {
+  def getMetricsReporters(): Option[String] = getOption(MetricsConfig.METRICS_REPORTERS)
+
+  def getMetricsFactoryClass(name: String): Option[String] = getOption(MetricsConfig.METRICS_REPORTER_FACTORY format name)
+
+  def getMetricsReporterStream(name: String): Option[String] = getOption(MetricsConfig.METRICS_SNAPSHOT_REPORTER_STREAM format name)
+
+  /**
+   * Returns a list of all metrics names from the config file. Useful for
+   * getting individual metrics.
+   */
+  def getMetricReporterNames() = {
+    getMetricsReporters match {
+      case Some(mr) => if (!"".equals(mr)) {
+        mr.split(",").map(name => name.trim).toList
+      } else {
+        List[String]()
+      }
+      case _ => List[String]()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/ScalaMapConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/ScalaMapConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/ScalaMapConfig.scala
new file mode 100644
index 0000000..5fb1f52
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/ScalaMapConfig.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+import org.apache.samza.SamzaException
+
+class ScalaMapConfig(config: Config) extends MapConfig(config) {
+  def getOrElse(k: String, els: String) = getOption(k).getOrElse(els)
+
+  def getOption(k: String): Option[String] = if (containsKey(k)) Some(config.get(k)) else None
+
+  def getExcept(k: String, msg: String = null): String = 
+    getOption(k) match {
+      case Some(s) => s
+      case _ =>
+        val error = 
+          if(msg == null) "Missing required configuration '%s'".format(k)
+          else "Missing required configuration '%s': %s".format(k, msg)
+        throw new SamzaException(error)
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/SerializerConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/SerializerConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/SerializerConfig.scala
new file mode 100644
index 0000000..0c026e6
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/SerializerConfig.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+import scala.collection.JavaConversions._
+
+object SerializerConfig {
+  // serializer config constants
+  val SERIALIZER_PREFIX = "serializers.registry.%s"
+  val SERDE = "serializers.registry.%s.class"
+
+  implicit def Config2Serializer(config: Config) = new SerializerConfig(config)
+}
+
+class SerializerConfig(config: Config) extends ScalaMapConfig(config) {
+  def getSerdeClass(name: String) = getOption(SerializerConfig.SERDE format name)
+
+  /**
+   * Returns a list of all serializer names from the config file. Useful for
+   * getting individual serializers.
+   */
+  def getSerdeNames() = {
+    val subConf = config.subset("serializers.registry.", true)
+    subConf.keys.filter(k => k.endsWith(".class")).map(_.replace(".class", ""))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala
new file mode 100644
index 0000000..fd8dab8
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+object ShellCommandConfig {
+  /**
+   * This environment variable is used to store a JSON serialized map of all configuration.
+   */
+  val ENV_CONFIG = "STREAMING_CONFIG"
+
+  /**
+   * A CSV list of partition IDs that a TaskRunner is responsible for (e.g. 0,2,4,6).
+   */
+  val ENV_PARTITION_IDS = "PARTITION_IDS"
+
+  /**
+   * A name for a TaskRunner.
+   */
+  val ENV_TASK_NAME = "TASK_NAME"
+
+  /**
+   * Arguments to be passed to the processing running the TaskRunner (or equivalent, for non JVM languages).
+   */
+  val ENV_SAMZA_OPTS = "SAMZA_OPTS"
+
+  val COMMAND_SHELL_EXECUTE = "task.execute"
+  val TASK_JVM_OPTS = "task.opts"
+
+  implicit def Config2ShellCommand(config: Config) = new ShellCommandConfig(config)
+}
+
+class ShellCommandConfig(config: Config) extends ScalaMapConfig(config) {
+  def getCommand = getOption(ShellCommandConfig.COMMAND_SHELL_EXECUTE).getOrElse("bin/run-task.sh")
+
+  def getTaskOpts = getOption(ShellCommandConfig.TASK_JVM_OPTS)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala
new file mode 100644
index 0000000..abcb034
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/StorageConfig.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import scala.collection.JavaConversions._
+
+import grizzled.slf4j.Logging
+
+object StorageConfig {
+  // stream config constants
+  val FACTORY = "stores.%s.factory"
+  val KEY_SERDE = "stores.%s.key.serde"
+  val MSG_SERDE = "stores.%s.msg.serde"
+  val CHANGELOG_STREAM = "stores.%s.changelog"
+
+  implicit def Config2Storage(config: Config) = new StorageConfig(config)
+}
+
+class StorageConfig(config: Config) extends ScalaMapConfig(config) with Logging {
+  import StorageConfig._
+  def getStorageFactoryClassName(name: String) = getOption(FACTORY.format(name))
+  def getStorageKeySerde(name: String) = getOption(StorageConfig.KEY_SERDE format name)
+  def getStorageMsgSerde(name: String) = getOption(StorageConfig.MSG_SERDE format name)
+  def getChangelogStream(name: String) = getOption(CHANGELOG_STREAM format name)
+  def getStoreNames: Seq[String] = {
+    val conf = config.subset("stores.", true)
+    conf.keys.filter(k => k.endsWith(".factory")).map(k => k.substring(0, k.length - ".factory".length)).toSeq
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
new file mode 100644
index 0000000..517e9ae
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import grizzled.slf4j.Logging
+import scala.collection.JavaConversions._
+import org.apache.samza.SamzaException
+import org.apache.samza.util.Util
+import org.apache.samza.system.SystemStream
+
+object StreamConfig {
+  // stream config constants
+  val STREAM_PREFIX = "systems.%s.streams.%s."
+  val MSG_SERDE = STREAM_PREFIX + "samza.msg.serde"
+  val KEY_SERDE = STREAM_PREFIX + "samza.key.serde"
+  val CONSUMER_RESET_OFFSET = STREAM_PREFIX + "samza.reset.offset"
+
+  implicit def Config2Stream(config: Config) = new StreamConfig(config)
+}
+
+class StreamConfig(config: Config) extends ScalaMapConfig(config) with Logging {
+  def getStreamMsgSerde(systemStream: SystemStream) =
+    getOption(StreamConfig.MSG_SERDE format (systemStream.getSystem, systemStream.getStream))
+
+  def getStreamKeySerde(systemStream: SystemStream) =
+    getOption(StreamConfig.KEY_SERDE format (systemStream.getSystem, systemStream.getStream))
+
+  def getResetOffsetMap(systemName: String) = {
+    val subConf = config.subset("systems.%s.streams." format systemName, true)
+    subConf
+      .filterKeys(k => k.endsWith(".samza.reset.offset"))
+      .map(kv => {
+        val streamName = kv._1.replace(".samza.reset.offset", "")
+        val systemStream = new SystemStream(systemName, streamName)
+        val resetVal = getResetOffset(systemStream)
+        (systemStream, resetVal)
+      }).toMap
+  }
+
+  def getResetOffset(systemStream: SystemStream) =
+    getOption(StreamConfig.CONSUMER_RESET_OFFSET format (systemStream.getSystem, systemStream.getStream)) match {
+      case Some("true") => true
+      case Some("false") => false
+      case Some(resetOffset) =>
+        warn("Got a configuration for %s that is not true, or false (was %s). Defaulting to false." format (StreamConfig.CONSUMER_RESET_OFFSET format (systemStream.getSystem, systemStream.getStream), resetOffset))
+        false
+      case _ => false
+    }
+
+  /**
+   * Returns a list of all SystemStreams that have a serde defined from the config file.
+   */
+  def getSerdeStreams(systemName: String) = {
+    val subConf = config.subset("systems.%s.streams." format systemName, true)
+    subConf
+      .keys
+      .filter(k => k.endsWith(".samza.msg.serde") || k.endsWith(".samza.key.serde"))
+      .map(k => {
+        val streamName = k.substring(0, k.length - 16 /* .samza.XXX.serde length */ )
+        new SystemStream(systemName, streamName)
+      }).toSet
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/SystemConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/SystemConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/SystemConfig.scala
new file mode 100644
index 0000000..ce63a8a
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/SystemConfig.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import scala.collection.JavaConversions._
+
+import grizzled.slf4j.Logging
+
+object SystemConfig {
+  // system config constants
+  val SYSTEM_PREFIX = "systems.%s."
+  val SYSTEM_FACTORY = "systems.%s.samza.factory"
+  val KEY_SERDE = "systems.%s.samza.key.serde"
+  val MSG_SERDE = "systems.%s.samza.msg.serde"
+
+  implicit def Config2System(config: Config) = new SystemConfig(config)
+}
+
+class SystemConfig(config: Config) extends ScalaMapConfig(config) with Logging {
+  def getSystemFactory(name: String) = getOption(SystemConfig.SYSTEM_FACTORY format name)
+
+  def getSystemKeySerde(name: String) = getOption(SystemConfig.KEY_SERDE format name)
+
+  def getSystemMsgSerde(name: String) = getOption(SystemConfig.MSG_SERDE format name)
+
+  /**
+   * Returns a list of all system names from the config file. Useful for
+   * getting individual systems.
+   */
+  def getSystemNames() = {
+    val subConf = config.subset("systems.", true)
+    // find all .samza.partition.manager keys, and strip the suffix
+    subConf.keys.filter(k => k.endsWith(".samza.factory")).map(_.replace(".samza.factory", ""))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala
new file mode 100644
index 0000000..0c742d8
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/TaskConfig.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config
+
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.SamzaException
+import org.apache.samza.util.Util
+import org.apache.samza.system.SystemStream
+
+object TaskConfig {
+  // task config constants
+  val INPUT_STREAMS = "task.inputs" // streaming.input-streams
+  val WINDOW_MS = "task.window.ms" // window period in milliseconds
+  val COMMIT_MS = "task.commit.ms" // commit period in milliseconds
+  val TASK_CLASS = "task.class" // streaming.task-factory-class
+  val COMMAND_BUILDER = "task.command.class" // streaming.task-factory-class
+  val LIFECYCLE_LISTENERS = "task.lifecycle.listeners" // li-generator,foo
+  val LIFECYCLE_LISTENER = "task.lifecycle.listener.%s.class" // task.lifecycle.listener.li-generator.class
+  val CHECKPOINT_MANAGER_FACTORY = "task.checkpoint.factory" // class name to use when sending offset checkpoints
+  val TASK_JMX_ENABLED = "task.jmx.enabled" // Start up a JMX server for this task?
+
+  implicit def Config2Task(config: Config) = new TaskConfig(config)
+}
+
+class TaskConfig(config: Config) extends ScalaMapConfig(config) {
+  def getInputStreams = getOption(TaskConfig.INPUT_STREAMS) match {
+    case Some(streams) => if (streams.length > 0) {
+      streams.split(",").map(systemStreamNames => {
+        Util.getSystemStreamFromNames(systemStreamNames)
+      }).toSet
+    } else {
+      Set[SystemStream]()
+    }
+    case _ => Set[SystemStream]()
+  }
+
+  def getWindowMs: Option[Long] = getOption(TaskConfig.WINDOW_MS) match {
+    case Some(ms) => Some(ms.toLong)
+    case _ => None
+  }
+
+  def getCommitMs: Option[Long] = getOption(TaskConfig.COMMIT_MS) match {
+    case Some(ms) => Some(ms.toLong)
+    case _ => None
+  }
+
+  def getLifecycleListeners(): Option[String] = getOption(TaskConfig.LIFECYCLE_LISTENERS)
+
+  def getLifecycleListenerClass(name: String): Option[String] = getOption(TaskConfig.LIFECYCLE_LISTENER format name)
+
+  def getTaskClass = getOption(TaskConfig.TASK_CLASS)
+
+  def getCommandClass = getOption(TaskConfig.COMMAND_BUILDER)
+
+  def getCheckpointManagerFactory() = getOption(TaskConfig.CHECKPOINT_MANAGER_FACTORY)
+
+  def getJmxServerEnabled = getBoolean(TaskConfig.TASK_JMX_ENABLED, true)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/factories/PropertiesConfigFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/factories/PropertiesConfigFactory.scala b/samza-core/src/main/scala/org/apache/samza/config/factories/PropertiesConfigFactory.scala
new file mode 100644
index 0000000..768cfa4
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/factories/PropertiesConfigFactory.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config.factories
+import java.io.FileInputStream
+import java.net.URI
+import java.util.Properties
+import scala.collection.JavaConversions._
+import org.apache.samza.config.Config
+import org.apache.samza.config.ConfigFactory
+import org.apache.samza.config.MapConfig
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+
+class PropertiesConfigFactory extends ConfigFactory with Logging {
+  def getConfig(configUri: URI): Config = {
+    val scheme = configUri.getScheme
+    if (scheme != null && !scheme.equals("file")) {
+      throw new SamzaException("only the file:// scheme is supported for properties files")
+    }
+
+    val configPath = configUri.getPath
+    val props = new Properties();
+    val in = new FileInputStream(configPath);
+
+    props.load(in);
+    in.close
+
+    debug("got config %s from config %s" format (props, configPath))
+
+    new MapConfig(props.toMap[String, String])
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/config/serializers/JsonConfigSerializer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/serializers/JsonConfigSerializer.scala b/samza-core/src/main/scala/org/apache/samza/config/serializers/JsonConfigSerializer.scala
new file mode 100644
index 0000000..60e65ea
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/config/serializers/JsonConfigSerializer.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.config.serializers
+import scala.collection.JavaConversions._
+
+import org.codehaus.jackson.map.ObjectMapper
+
+import org.apache.samza.config.Config
+import org.apache.samza.config.MapConfig
+
+import java.util.HashMap
+
+object JsonConfigSerializer {
+  val jsonMapper = new ObjectMapper()
+
+  def fromJson(string: String): Config = {
+    val map = jsonMapper.readValue(string, classOf[HashMap[String, String]])
+    new MapConfig(map)
+  }
+
+  def toJson(config: Config) = jsonMapper.writeValueAsString(new HashMap[String, String](config))
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
new file mode 100644
index 0000000..2243b5c
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
@@ -0,0 +1,608 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.container
+
+import java.io.File
+import grizzled.slf4j.Logging
+import org.apache.samza.Partition
+import org.apache.samza.SamzaException
+import org.apache.samza.checkpoint.CheckpointManager
+import org.apache.samza.checkpoint.CheckpointManagerFactory
+import org.apache.samza.config.Config
+import org.apache.samza.config.MetricsConfig.Config2Metrics
+import org.apache.samza.config.SerializerConfig.Config2Serializer
+import org.apache.samza.config.ShellCommandConfig
+import org.apache.samza.config.StorageConfig.Config2Storage
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.TaskConfig.Config2Task
+import org.apache.samza.config.serializers.JsonConfigSerializer
+import org.apache.samza.metrics.JmxServer
+import org.apache.samza.metrics.JvmMetrics
+import org.apache.samza.metrics.MetricsReporter
+import org.apache.samza.metrics.MetricsReporterFactory
+import org.apache.samza.serializers.Serde
+import org.apache.samza.serializers.SerdeFactory
+import org.apache.samza.serializers.SerdeManager
+import org.apache.samza.storage.StorageEngineFactory
+import org.apache.samza.storage.TaskStorageManager
+import org.apache.samza.system.SystemFactory
+import org.apache.samza.system.SystemStream
+import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.task.StreamTask
+import org.apache.samza.task.TaskLifecycleListener
+import org.apache.samza.task.TaskLifecycleListenerFactory
+import org.apache.samza.util.Util
+import org.apache.samza.task.ReadableCoordinator
+import org.apache.samza.system.SystemProducers
+import org.apache.samza.task.ReadableCollector
+import org.apache.samza.system.DefaultPicker
+import org.apache.samza.system.SystemConsumers
+
+object SamzaContainer extends Logging {
+  def main(args: Array[String]) {
+    val jmxServer = new JmxServer
+    val containerName = System.getenv(ShellCommandConfig.ENV_TASK_NAME)
+    val configStr = System.getenv(ShellCommandConfig.ENV_CONFIG)
+    val config = JsonConfigSerializer.fromJson(configStr)
+    val partitionIdsCsv = System.getenv(ShellCommandConfig.ENV_PARTITION_IDS)
+    val partitions = if (partitionIdsCsv.length > 0) {
+      partitionIdsCsv.split(",")
+        .map(partitionIdStr => new Partition(partitionIdStr.toInt))
+        .toSet
+    } else {
+      throw new SamzaException("No partitions for this task. Can't run a task without partition assignments. It's likely that the partition manager for this system doesn't know about the stream you're trying to read.")
+    }
+
+    try {
+      SamzaContainer(containerName, partitions, config).run
+    } finally {
+      jmxServer.stop
+    }
+  }
+
+  def apply(containerName: String, partitions: Set[Partition], config: Config) = {
+    info("Setting up Samza container: %s" format containerName)
+    info("Using partitions: %s" format partitions)
+    info("Using configuration: %s" format config)
+
+    val samzaContainerMetrics = new SamzaContainerMetrics(containerName)
+
+    val inputStreams = config.getInputStreams
+    val inputSystems = inputStreams.map(_.getSystem)
+
+    info("Got input streams: %s" format inputStreams)
+
+    val systemNames = config.getSystemNames
+
+    info("Got system names: %s" format systemNames)
+
+    val resetInputStreams = systemNames.flatMap(systemName => {
+      config.getResetOffsetMap(systemName)
+    }).toMap
+
+    info("Got input stream resets: %s" format resetInputStreams)
+
+    val serdeStreams = systemNames.foldLeft(Set[SystemStream]())(_ ++ config.getSerdeStreams(_))
+
+    debug("Got serde streams: %s" format serdeStreams)
+
+    val serdeNames = config.getSerdeNames
+
+    info("Got serde names: %s" format serdeNames)
+
+    val systemFactories = systemNames.map(systemName => {
+      val systemFactoryClassName = config
+        .getSystemFactory(systemName)
+        .getOrElse(throw new SamzaException("A stream uses system %s, which is missing from the configuration." format systemName))
+      (systemName, Util.getObj[SystemFactory](systemFactoryClassName))
+    }).toMap
+
+    info("Got system factories: %s" format systemFactories.keys)
+
+    val consumers = inputSystems
+      .map(systemName => {
+        val systemFactory = systemFactories(systemName)
+
+        try {
+          (systemName, systemFactory.getConsumer(systemName, config, samzaContainerMetrics.registry))
+        } catch {
+          case e: Throwable =>
+            info("Failed to create a consumer for %s, so skipping." format systemName)
+            debug(e)
+            (systemName, null)
+        }
+      })
+      .filter(_._2 != null)
+      .toMap
+
+    info("Got system consumers: %s" format consumers.keys)
+
+    val producers = systemFactories
+      .map {
+        case (systemName, systemFactory) =>
+          try {
+            (systemName, systemFactory.getProducer(systemName, config, samzaContainerMetrics.registry))
+          } catch {
+            case e: Throwable =>
+              info("Failed to create a producer for %s, so skipping." format systemName)
+              debug(e)
+              (systemName, null)
+          }
+      }
+      .filter(_._2 != null)
+      .toMap
+
+    info("Got system producers: %s" format producers.keys)
+
+    val serdes = serdeNames.map(serdeName => {
+      val serdeClassName = config
+        .getSerdeClass(serdeName)
+        .getOrElse(throw new SamzaException("No class defined for serde: %s." format serdeName))
+
+      val serde = Util.getObj[SerdeFactory[Object]](serdeClassName)
+        .getSerde(serdeName, config)
+
+      (serdeName, serde)
+    }).toMap
+
+    info("Got serdes: %s" format serdes.keys)
+
+    /*
+     * A Helper function to build a Map[String, Serde] (systemName -> Serde) for systems defined in the config. This is useful to build both key and message serde maps.
+     */
+    val buildSystemSerdeMap = (getSerdeName: (String) => Option[String]) => {
+      systemNames
+        .filter(getSerdeName(_).isDefined)
+        .map(systemName => {
+          val serdeName = getSerdeName(systemName).get
+          val serde = serdes.getOrElse(serdeName, throw new SamzaException("Serde %s for system %s does not exist in configuration." format (serdeName, systemName)))
+          (systemName, serde)
+        }).toMap
+    }
+
+    /*
+     * A Helper function to build a Map[SystemStream, Serde] for streams defined in the config. This is useful to build both key and message serde maps.
+     */
+    val buildSystemStreamSerdeMap = (getSerdeName: (SystemStream) => Option[String]) => {
+      (serdeStreams ++ inputStreams)
+        .filter(systemStream => getSerdeName(systemStream).isDefined)
+        .map(systemStream => {
+          val serdeName = getSerdeName(systemStream).get
+          val serde = serdes.getOrElse(serdeName, throw new SamzaException("Serde %s for system %s does not exist in configuration." format (serdeName, systemStream)))
+          (systemStream, serde)
+        }).toMap
+    }
+
+    val systemKeySerdes = buildSystemSerdeMap((systemName: String) => config.getSystemKeySerde(systemName))
+
+    debug("Got system key serdes: %s" format systemKeySerdes)
+
+    val systemMessageSerdes = buildSystemSerdeMap((systemName: String) => config.getSystemMsgSerde(systemName))
+
+    debug("Got system message serdes: %s" format systemMessageSerdes)
+
+    val systemStreamKeySerdes = buildSystemStreamSerdeMap((systemStream: SystemStream) => config.getStreamKeySerde(systemStream))
+
+    debug("Got system stream key serdes: %s" format systemStreamKeySerdes)
+
+    val systemStreamMessageSerdes = buildSystemStreamSerdeMap((systemStream: SystemStream) => config.getStreamMsgSerde(systemStream))
+
+    debug("Got system stream message serdes: %s" format systemStreamMessageSerdes)
+
+    val changeLogSystemStreams = config
+      .getStoreNames
+      .filter(config.getChangelogStream(_).isDefined)
+      .map(name => (name, config.getChangelogStream(name).get)).toMap
+      .mapValues(Util.getSystemStreamFromNames(_))
+
+    info("Got change log system streams: %s" format changeLogSystemStreams)
+
+    val serdeManager = new SerdeManager(
+      serdes = serdes,
+      systemKeySerdes = systemKeySerdes,
+      systemMessageSerdes = systemMessageSerdes,
+      systemStreamKeySerdes = systemStreamKeySerdes,
+      systemStreamMessageSerdes = systemStreamMessageSerdes,
+      changeLogSystemStreams = changeLogSystemStreams.values.toSet)
+
+    info("Setting up JVM metrics.")
+
+    val jvm = new JvmMetrics(samzaContainerMetrics.registry)
+
+    info("Setting up incoming message envelope picker.")
+
+    val picker = new DefaultPicker
+
+    info("Setting up metrics reporters.")
+
+    val reporters = config.getMetricReporterNames.map(reporterName => {
+      val metricsFactoryClassName = config
+        .getMetricsFactoryClass(reporterName)
+        .getOrElse(throw new SamzaException("Metrics reporter %s missing .class config" format reporterName))
+
+      val reporter =
+        Util
+          .getObj[MetricsReporterFactory](metricsFactoryClassName)
+          .getMetricsReporter(reporterName, containerName, config)
+      (reporterName, reporter)
+    }).toMap
+
+    info("Got metrics reporters: %s" format reporters.keys)
+
+    val checkpointManager = config.getCheckpointManagerFactory match {
+      case Some(checkpointFactoryClassName) =>
+        Util
+          .getObj[CheckpointManagerFactory](checkpointFactoryClassName)
+          .getCheckpointManager(config, samzaContainerMetrics.registry)
+      case _ => null
+    }
+
+    info("Got checkpoint manager: %s" format checkpointManager)
+
+    val consumerMultiplexer = new SystemConsumers(
+      // TODO add config values for no new message timeout and max msgs per stream partition
+      picker = picker,
+      consumers = consumers,
+      serdeManager = serdeManager)
+
+    val producerMultiplexer = new SystemProducers(
+      producers = producers,
+      serdeManager = serdeManager)
+
+    val listeners = config.getLifecycleListeners match {
+      case Some(listeners) => {
+        listeners.split(",").map(listenerName => {
+          info("Loading lifecycle listener: %s" format listenerName)
+
+          val listenerClassName = config.getLifecycleListenerClass(listenerName).getOrElse(throw new SamzaException("Referencing missing listener %s in config" format listenerName))
+
+          Util.getObj[TaskLifecycleListenerFactory](listenerClassName)
+            .getLifecyleListener(listenerName, config)
+        }).toList
+      }
+      case _ => {
+        info("No lifecycle listeners found")
+
+        List[TaskLifecycleListener]()
+      }
+    }
+
+    // TODO not sure how we should make this config based, or not. Kind of 
+    // strange, since it has some dynamic directories when used with YARN.
+    val storeBaseDir = new File(System.getProperty("user.dir"), "state")
+
+    info("Got storage engine base directory: %s" format storeBaseDir)
+
+    val storageEngineFactories = config
+      .getStoreNames
+      .map(storeName => {
+        val storageFactoryClassName = config
+          .getStorageFactoryClassName(storeName)
+          .getOrElse(throw new SamzaException("Missing storage factory for %s." format storeName))
+        (storeName, Util.getObj[StorageEngineFactory[Object, Object]](storageFactoryClassName))
+      }).toMap
+
+    info("Got storage engines: %s" format storageEngineFactories.keys)
+
+    val taskClassName = config
+      .getTaskClass
+      .getOrElse(throw new SamzaException("No task class defined in configuration."))
+
+    info("Got stream task class: %s" format taskClassName)
+
+    val taskWindowMs = config.getWindowMs.getOrElse(-1L)
+
+    info("Got window milliseconds: %s" format taskWindowMs)
+
+    val taskCommitMs = config.getCommitMs.getOrElse(60000L)
+
+    info("Got commit milliseconds: %s" format taskCommitMs)
+
+    // Wire up all task-level (unshared) objects.
+    val taskInstances = partitions.map(partition => {
+      debug("Setting up task instance: %s" format partition)
+
+      val task = Util.getObj[StreamTask](taskClassName)
+
+      val collector = new ReadableCollector
+
+      val taskInstanceMetrics = new TaskInstanceMetrics(partition)
+
+      val storeConsumers = changeLogSystemStreams
+        .map {
+          case (storeName, changeLogSystemStream) =>
+            val systemConsumer = systemFactories
+              .getOrElse(changeLogSystemStream.getSystem, throw new SamzaException("Changelog system %s for store %s does not exist in the config." format (changeLogSystemStream, storeName)))
+              .getConsumer(changeLogSystemStream.getSystem, config, taskInstanceMetrics.registry)
+            (storeName, systemConsumer)
+        }.toMap
+
+      info("Got store consumers: %s" format storeConsumers)
+
+      val taskStores = storageEngineFactories
+        .map {
+          case (storeName, storageEngineFactory) =>
+            val changeLogSystemStreamPartition = if (changeLogSystemStreams.contains(storeName)) {
+              new SystemStreamPartition(changeLogSystemStreams(storeName), partition)
+            } else {
+              null
+            }
+            val keySerde = config.getStorageKeySerde(storeName) match {
+              case Some(keySerde) => serdes(keySerde)
+              case _ => null
+            }
+            val msgSerde = config.getStorageMsgSerde(storeName) match {
+              case Some(msgSerde) => serdes(msgSerde)
+              case _ => null
+            }
+            val storePartitionDir = TaskStorageManager.getStorePartitionDir(storeBaseDir, storeName, partition)
+            val storageEngine = storageEngineFactory.getStorageEngine(
+              storeName,
+              storePartitionDir,
+              keySerde,
+              msgSerde,
+              collector,
+              config,
+              taskInstanceMetrics.registry,
+              changeLogSystemStreamPartition)
+            (storeName, storageEngine)
+        }
+
+      info("Got task stores: %s" format taskStores)
+
+      val storageManager = new TaskStorageManager(
+        partition = partition,
+        taskStores = taskStores,
+        storeConsumers = storeConsumers,
+        changeLogSystemStreams = changeLogSystemStreams,
+        storeBaseDir = storeBaseDir)
+
+      val taskInstance = new TaskInstance(
+        task = task,
+        partition = partition,
+        config = config,
+        metrics = taskInstanceMetrics,
+        consumerMultiplexer = consumerMultiplexer,
+        producerMultiplexer = producerMultiplexer,
+        storageManager = storageManager,
+        checkpointManager = checkpointManager,
+        reporters = reporters,
+        listeners = listeners,
+        inputStreams = inputStreams,
+        resetInputStreams = resetInputStreams,
+        windowMs = taskWindowMs,
+        commitMs = taskCommitMs,
+        collector = collector)
+
+      (partition, taskInstance)
+    }).toMap
+
+    info("Samza container setup complete.")
+
+    new SamzaContainer(
+      taskInstances = taskInstances,
+      config = config,
+      consumerMultiplexer = consumerMultiplexer,
+      producerMultiplexer = producerMultiplexer,
+      checkpointManager = checkpointManager,
+      metrics = samzaContainerMetrics,
+      reporters = reporters,
+      jvm = jvm)
+  }
+}
+
+class SamzaContainer(
+  taskInstances: Map[Partition, TaskInstance],
+  config: Config,
+  consumerMultiplexer: SystemConsumers,
+  producerMultiplexer: SystemProducers,
+  checkpointManager: CheckpointManager = null,
+  metrics: SamzaContainerMetrics = new SamzaContainerMetrics,
+  reporters: Map[String, MetricsReporter] = Map(),
+  jvm: JvmMetrics = null) extends Runnable with Logging {
+
+  def run {
+    info("Entering run loop.")
+
+    startMetrics
+    startCheckpoints
+    startStores
+    startTask
+    startProducers
+    startConsumers
+
+    try {
+      while (true) {
+        val coordinator = new ReadableCoordinator
+
+        process(coordinator)
+        window(coordinator)
+        send
+        commit(coordinator)
+
+        if (coordinator.shutdownRequested) {
+          info("Shutdown requested.")
+
+          return
+        }
+      }
+    } catch {
+      case e: Throwable =>
+        error("Caught exception in process loop.", e)
+        throw e
+    } finally {
+      info("Shutting down.")
+
+      shutdownConsumers
+      shutdownProducers
+      shutdownTask
+      shutdownStores
+      shutdownCheckpoints
+      shutdownMetrics
+
+      info("Shutdown complete.")
+    }
+  }
+
+  def startMetrics {
+    info("Registering task instances with metrics.")
+
+    taskInstances.values.foreach(_.registerMetrics)
+
+    info("Starting JVM metrics.")
+
+    if (jvm != null) {
+      jvm.start
+    }
+
+    info("Starting metrics reporters.")
+
+    reporters.values.foreach(reporter => {
+      reporter.register(metrics.source, metrics.registry)
+      reporter.start
+    })
+  }
+
+  def startCheckpoints {
+    info("Registering task instances with checkpoints.")
+
+    taskInstances.values.foreach(_.registerCheckpoints)
+
+    if (checkpointManager != null) {
+      info("Registering checkpoint manager.")
+
+      checkpointManager.start
+    } else {
+      warn("No checkpoint manager defined. No consumer offsets will be maintained for this job.")
+    }
+  }
+
+  def startStores {
+    info("Starting task instance stores.")
+
+    taskInstances.values.foreach(_.startStores)
+  }
+
+  def startTask {
+    info("Initializing stream tasks.")
+
+    taskInstances.values.foreach(_.initTask)
+  }
+
+  def startProducers {
+    info("Registering task instances with producers.")
+
+    taskInstances.values.foreach(_.registerProducers)
+
+    info("Starting producer multiplexer.")
+
+    producerMultiplexer.start
+  }
+
+  def startConsumers {
+    info("Registering task instances with consumers.")
+
+    taskInstances.values.foreach(_.registerConsumers)
+
+    info("Starting consumer multiplexer.")
+
+    consumerMultiplexer.start
+  }
+
+  def process(coordinator: ReadableCoordinator) {
+    trace("Attempting to pick a message to process.")
+
+    val envelope = consumerMultiplexer.pick
+
+    if (envelope != null) {
+      val partition = envelope.getSystemStreamPartition.getPartition
+
+      trace("Processing incoming message envelope for partition %s." format partition)
+
+      taskInstances(partition).process(envelope, coordinator)
+    } else {
+      trace("No incoming message envelope was available.")
+    }
+  }
+
+  def window(coordinator: ReadableCoordinator) {
+    trace("Windowing stream tasks.")
+
+    taskInstances.values.foreach(_.window(coordinator))
+  }
+
+  def send {
+    trace("Triggering send in task instances.")
+
+    taskInstances.values.foreach(_.send)
+  }
+
+  def commit(coordinator: ReadableCoordinator) {
+    trace("Committing task instances.")
+
+    taskInstances.values.foreach(_.commit(coordinator))
+  }
+
+  def shutdownConsumers {
+    info("Shutting down consumer multiplexer.")
+
+    consumerMultiplexer.stop
+  }
+
+  def shutdownProducers {
+    info("Shutting down producer multiplexer.")
+
+    producerMultiplexer.stop
+  }
+
+  def shutdownTask {
+    info("Shutting down task instance stream tasks.")
+
+    taskInstances.values.foreach(_.shutdownTask)
+  }
+
+  def shutdownStores {
+    info("Shutting down task instance stores.")
+
+    taskInstances.values.foreach(_.shutdownStores)
+  }
+
+  def shutdownCheckpoints {
+    if (checkpointManager != null) {
+      info("Shutting down checkpoint manager.")
+      checkpointManager.stop
+    } else {
+      info("No checkpoint manager defined, so skipping checkpoint manager stop.")
+    }
+  }
+
+  def shutdownMetrics {
+    info("Shutting down metrics reporters.")
+
+    reporters.values.foreach(_.stop)
+
+    if (jvm != null) {
+      info("Shutting down JVM metrics.")
+
+      jvm.stop
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala
new file mode 100644
index 0000000..81cf356
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.container
+
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.metrics.MetricsRegistryMap
+
+class SamzaContainerMetrics(
+  val containerName: String = "unnamed-container",
+  val registry: ReadableMetricsRegistry = new MetricsRegistryMap) {
+
+  val source = containerName
+  val commits = registry.newCounter("samza.task.SamzaContainer", "commits")
+
+  // TODO .. etc ..
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
new file mode 100644
index 0000000..5fc9316
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.container
+
+import org.apache.samza.metrics.MetricsReporter
+import org.apache.samza.config.Config
+import org.apache.samza.Partition
+import grizzled.slf4j.Logging
+import scala.collection.JavaConversions._
+import org.apache.samza.storage.TaskStorageManager
+import org.apache.samza.config.StreamConfig.Config2Stream
+import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.task.TaskContext
+import org.apache.samza.task.ClosableTask
+import org.apache.samza.task.InitableTask
+import org.apache.samza.system.IncomingMessageEnvelope
+import org.apache.samza.task.WindowableTask
+import org.apache.samza.checkpoint.CheckpointManager
+import org.apache.samza.task.TaskLifecycleListener
+import org.apache.samza.task.StreamTask
+import org.apache.samza.system.SystemStream
+import org.apache.samza.checkpoint.Checkpoint
+import org.apache.samza.task.ReadableCollector
+import org.apache.samza.system.SystemConsumers
+import org.apache.samza.system.SystemProducers
+import org.apache.samza.task.ReadableCoordinator
+
+class TaskInstance(
+  task: StreamTask,
+  partition: Partition,
+  config: Config,
+  metrics: TaskInstanceMetrics,
+  consumerMultiplexer: SystemConsumers,
+  producerMultiplexer: SystemProducers,
+  storageManager: TaskStorageManager = null,
+  checkpointManager: CheckpointManager = null,
+  reporters: Map[String, MetricsReporter] = Map(),
+  listeners: Seq[TaskLifecycleListener] = Seq(),
+  inputStreams: Set[SystemStream] = Set(),
+  resetInputStreams: Map[SystemStream, Boolean] = Map(),
+  queueSize: Int = 1000,
+  windowMs: Long = -1,
+  commitMs: Long = 60000,
+  clock: () => Long = { System.currentTimeMillis },
+  collector: ReadableCollector = new ReadableCollector) extends Logging {
+
+  var offsets = Map[SystemStream, String]()
+  var lastWindowMs = 0L
+  var lastCommitMs = 0L
+  val isInitableTask = task.isInstanceOf[InitableTask]
+  val isWindowableTask = task.isInstanceOf[WindowableTask]
+  val isClosableTask = task.isInstanceOf[ClosableTask]
+  val context = new TaskContext {
+    def getMetricsRegistry = metrics.registry
+    def getPartition = partition
+    def getStore(storeName: String) = if (storageManager != null) {
+      storageManager(storeName)
+    } else {
+      warn("No store found for name: %s" format storeName)
+
+      null
+    }
+  }
+
+  def registerMetrics {
+    debug("Registering metrics for partition: %s." format partition)
+
+    reporters.values.foreach(_.register(metrics.source, metrics.registry))
+  }
+
+  def registerCheckpoints {
+    if (checkpointManager != null) {
+      debug("Registering checkpoint manager for partition: %s." format partition)
+
+      checkpointManager.register(partition)
+    } else {
+      debug("Skipping checkpoint manager registration for partition: %s." format partition)
+    }
+  }
+
+  def startStores {
+    if (storageManager != null) {
+      debug("Starting storage manager for partition: %s." format partition)
+
+      storageManager.init(collector)
+    } else {
+      debug("Skipping storage manager initialization for partition: %s." format partition)
+    }
+  }
+
+  def initTask {
+    listeners.foreach(_.beforeInit(config, context))
+
+    if (isInitableTask) {
+      debug("Initializing task for partition: %s." format partition)
+
+      task.asInstanceOf[InitableTask].init(config, context)
+    } else {
+      debug("Skipping task initialization for partition: %s." format partition)
+    }
+
+    listeners.foreach(_.afterInit(config, context))
+  }
+
+  def registerProducers {
+    debug("Registering producers for partition: %s." format partition)
+
+    producerMultiplexer.register(metrics.source)
+  }
+
+  def registerConsumers {
+    if (checkpointManager != null) {
+      debug("Loading checkpoints for partition: %s." format partition)
+
+      val checkpoint = checkpointManager.readLastCheckpoint(partition)
+
+      if (checkpoint != null) {
+        for ((systemStream, offset) <- checkpoint.getOffsets) {
+          if (!resetInputStreams.getOrElse(systemStream, false)) {
+            offsets += systemStream -> offset
+          } else {
+            info("Got offset %s for %s, but ignoring, since stream was configured to reset offsets." format (offset, systemStream))
+          }
+        }
+
+        info("Successfully loaded offsets for partition: %s, %s" format (partition, offsets))
+      } else {
+        warn("No checkpoint found for partition: %s. This is allowed if this is your first time running the job, but if it's not, you've probably lost data." format partition)
+      }
+    }
+
+    debug("Registering consumers for partition: %s." format partition)
+
+    inputStreams.foreach(stream =>
+      consumerMultiplexer.register(
+        new SystemStreamPartition(stream, partition),
+        offsets.get(stream).getOrElse(null)))
+  }
+
+  def process(envelope: IncomingMessageEnvelope, coordinator: ReadableCoordinator) {
+    listeners.foreach(_.beforeProcess(envelope, config, context))
+
+    trace("Processing incoming message envelope for partition: %s, %s" format (partition, envelope.getSystemStreamPartition))
+
+    task.process(envelope, collector, coordinator)
+
+    listeners.foreach(_.afterProcess(envelope, config, context))
+
+    trace("Updating offset map for partition: %s, %s, %s" format (partition, envelope.getSystemStreamPartition, envelope.getOffset))
+
+    offsets += envelope.getSystemStreamPartition -> envelope.getOffset
+  }
+
+  def window(coordinator: ReadableCoordinator) {
+    if (isWindowableTask && windowMs >= 0 && lastWindowMs + windowMs < clock()) {
+      trace("Windowing for partition: %s" format partition)
+
+      task.asInstanceOf[WindowableTask].window(collector, coordinator)
+      lastWindowMs = clock()
+
+      trace("Assigned last window time for partition: %s, %s" format (partition, lastWindowMs))
+    } else {
+      trace("Skipping window for partition: %s" format partition)
+    }
+  }
+
+  def send {
+    if (collector.envelopes.size > 0) {
+      trace("Sending messages for partition: %s, %s" format (partition, collector.envelopes.size))
+
+      collector.envelopes.foreach(envelope => producerMultiplexer.send(metrics.source, envelope))
+
+      trace("Resetting collector for partition: %s" format partition)
+
+      collector.reset
+    } else {
+      trace("Skipping send for partition %s because no messages were collected." format partition)
+    }
+  }
+
+  def commit(coordinator: ReadableCoordinator) {
+    if (lastCommitMs + commitMs < clock() || coordinator.isCommitRequested || coordinator.isShutdownRequested) {
+      trace("Flushing state stores for partition: %s" format partition)
+
+      storageManager.flush
+
+      trace("Committing producers for partition: %s" format partition)
+
+      producerMultiplexer.commit(metrics.source)
+
+      if (checkpointManager != null) {
+        trace("Committing checkpoint manager for partition: %s" format partition)
+
+        checkpointManager.writeCheckpoint(partition, new Checkpoint(offsets))
+      }
+
+      lastCommitMs = clock()
+    }
+  }
+
+  def shutdownTask {
+    listeners.foreach(_.beforeClose(config, context))
+
+    if (task.isInstanceOf[ClosableTask]) {
+      debug("Shutting down stream task for partition: %s" format partition)
+
+      task.asInstanceOf[ClosableTask].close
+    } else {
+      debug("Skipping stream task shutdown for partition: %s" format partition)
+    }
+
+    listeners.foreach(_.afterClose(config, context))
+  }
+
+  def shutdownStores {
+    if (storageManager != null) {
+      debug("Shutting down storage manager for partition: %s" format partition)
+
+      storageManager.stop
+    } else {
+      debug("Skipping storage manager shutdown for partition: %s" format partition)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala
new file mode 100644
index 0000000..07d72c7
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.container
+
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.metrics.MetricsRegistryMap
+import org.apache.samza.Partition
+
+class TaskInstanceMetrics(
+  val partition: Partition,
+  val registry: ReadableMetricsRegistry = new MetricsRegistryMap) {
+
+  val source = "Partition-%s" format partition.getPartitionId
+  val commits = registry.newCounter("samza.task.TaskInstance", "commits")
+
+  // TODO .. etc ..
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
new file mode 100644
index 0000000..f3a75af
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/job/JobRunner.scala
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job
+
+import java.lang.String
+import java.net.URI
+import org.apache.samza.job.ApplicationStatus.Running
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.config._
+import grizzled.slf4j.Logging
+import joptsimple.OptionParser
+import joptsimple.util.KeyValuePair
+import org.apache.samza.SamzaException
+import org.apache.samza.config.factories.PropertiesConfigFactory
+import scala.Some
+import org.apache.samza.util.Util
+import scala.collection.mutable.Buffer
+import scala.collection.JavaConversions._
+
+object JobRunner extends Logging {
+  def main(args: Array[String]) {
+    // Define parameters.
+    var parser = new OptionParser()
+    val configFactoryOpt = 
+      parser.accepts("config-factory", "The config factory to use to read your config file.")
+            .withRequiredArg
+            .ofType(classOf[java.lang.String])
+            .describedAs("com.foo.bar.ClassName")
+            .defaultsTo(classOf[PropertiesConfigFactory].getName)
+    val configPathOpt =
+      parser.accepts("config-path", "URI location to a config file (e.g. file:///some/local/path.properties). " + 
+                                    "If multiple files are given they are all used with later files overriding any values that appear in earlier files.")
+            .withRequiredArg
+            .ofType(classOf[URI])
+            .describedAs("path")
+    val configOverrideOpt = 
+      parser.accepts("config", "A configuration value in the form key=value. Command line properties override any configuration values given.")
+            .withRequiredArg
+            .ofType(classOf[KeyValuePair])
+            .describedAs("key=value")
+    var options = parser.parse(args: _*)
+
+    // Verify legitimate parameters.
+    if (!options.has(configPathOpt)) {
+      parser.printHelpOn(System.err)
+      System.exit(-1)
+    }
+
+    // Set up the job parameters.
+    val configFactoryClass = options.valueOf(configFactoryOpt)
+    val configPaths = options.valuesOf(configPathOpt)
+    val configFactory = Class.forName(configFactoryClass).newInstance.asInstanceOf[ConfigFactory]
+    val configOverrides = options.valuesOf(configOverrideOpt).map(kv => (kv.key, kv.value)).toMap
+    
+    val configs: Buffer[java.util.Map[String, String]] = configPaths.map(configFactory.getConfig)
+    configs += configOverrides
+
+    new JobRunner(new MapConfig(configs)).run
+  }
+}
+
+/**
+ * ConfigRunner is a helper class that sets up and executes a Samza job based
+ * on a config URI. The configFactory is instantiated, fed the configPath,
+ * and returns a Config, which is used to execute the job.
+ */
+class JobRunner(config: Config) extends Logging with Runnable {
+
+  def run() {
+    val conf = rewriteConfig(config)
+
+    val jobFactoryClass = conf.getStreamJobFactoryClass match {
+      case Some(factoryClass) => factoryClass
+      case _ => throw new SamzaException("no job factory class defined")
+    }
+
+    val jobFactory = Class.forName(jobFactoryClass).newInstance.asInstanceOf[StreamJobFactory]
+
+    info("job factory: %s" format (jobFactoryClass))
+    debug("config: %s" format (conf))
+
+    // Create the actual job, and submit it.
+    val job = jobFactory.getJob(conf).submit
+
+    info("waiting for job to start")
+
+    // Wait until the job has started, then exit.
+    Option(job.waitForStatus(Running, 500)) match {
+      case Some(appStatus) => {
+        if (Running.equals(appStatus)) {
+          info("job started successfully")
+        } else {
+          warn("unable to start job successfully. job has status %s" format (appStatus))
+        }
+      }
+      case _ => warn("unable to start job successfully.")
+    }
+
+    info("exiting")
+  }
+
+  // Apply any and all config re-writer classes that the user has specified
+  def rewriteConfig(config: Config): Config = {
+    def rewrite(c: Config, rewriterName: String): Config = {
+      val klass = config
+        .getConfigRewriterClass(rewriterName)
+        .getOrElse(throw new SamzaException("Unable to find class config for config rewriter %s." format rewriterName))
+      val rewriter = Util.getObj[ConfigRewriter](klass)
+      info("Re-writing config file with " + rewriter)
+      rewriter.rewrite(rewriterName, c)
+    }
+
+    config.getConfigRewriters match {
+      case Some(rewriters) => rewriters.split(",").foldLeft(config)(rewrite(_, _))
+      case None => config
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/job/ShellCommandBuilder.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/ShellCommandBuilder.scala b/samza-core/src/main/scala/org/apache/samza/job/ShellCommandBuilder.scala
new file mode 100644
index 0000000..f55ca4c
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/job/ShellCommandBuilder.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job
+
+import scala.collection.JavaConversions._
+import org.apache.samza.config.ShellCommandConfig
+import org.apache.samza.config.ShellCommandConfig.Config2ShellCommand
+import org.apache.samza.config.serializers.JsonConfigSerializer
+
+class ShellCommandBuilder extends CommandBuilder {
+  def buildCommand() = config.getCommand
+
+  def buildEnvironment(): java.util.Map[String, String] = {
+    val parts = if (partitions.size() > 0) partitions.map(_.getPartitionId.toString).reduceLeft(_ + "," + _) else ""
+
+    Map(
+      ShellCommandConfig.ENV_TASK_NAME -> name,
+      ShellCommandConfig.ENV_PARTITION_IDS -> parts,
+      ShellCommandConfig.ENV_CONFIG -> JsonConfigSerializer.toJson(config),
+      ShellCommandConfig.ENV_SAMZA_OPTS -> config.getTaskOpts.getOrElse(""))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/job/local/LocalJobFactory.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/LocalJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/LocalJobFactory.scala
new file mode 100644
index 0000000..ddb119b
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/job/local/LocalJobFactory.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.local
+import org.apache.samza.config.TaskConfig._
+import org.apache.samza.config.Config
+import org.apache.samza.config.SystemConfig._
+import org.apache.samza.config.StreamConfig._
+import org.apache.samza.config.ShellCommandConfig._
+import org.apache.samza.job.CommandBuilder
+import org.apache.samza.job.StreamJob
+import org.apache.samza.job.StreamJobFactory
+import scala.collection.JavaConversions._
+import org.apache.samza.Partition
+import grizzled.slf4j.Logging
+import org.apache.samza.SamzaException
+import org.apache.samza.container.SamzaContainer
+import org.apache.samza.util.Util
+import org.apache.samza.job.ShellCommandBuilder
+
+class LocalJobFactory extends StreamJobFactory with Logging {
+  def getJob(config: Config): StreamJob = {
+    val taskName = "local-task"
+    val partitions = Util.getMaxInputStreamPartitions(config)
+
+    info("got partitions for job %s" format partitions)
+
+    if (partitions.size <= 0) {
+      throw new SamzaException("No partitions were detected for your input streams. It's likely that the system(s) specified don't know about the input streams: %s" format config.getInputStreams)
+    }
+
+    config.getCommandClass match {
+      case Some(cmdBuilderClassName) => {
+        // A command class was specified, so we need to use a process job to
+        // execute the command in its own process.
+        val cmdBuilder = Class.forName(cmdBuilderClassName).newInstance.asInstanceOf[CommandBuilder]
+
+        cmdBuilder
+          .setConfig(config)
+          .setName(taskName)
+          .setPartitions(partitions)
+
+        val processBuilder = new ProcessBuilder(cmdBuilder.buildCommand.split(" ").toList)
+
+        processBuilder
+          .environment
+          .putAll(cmdBuilder.buildEnvironment)
+
+        new ProcessJob(processBuilder)
+      }
+      case _ => {
+        info("No config specified for %s. Defaulting to ThreadJob, which is only meant for debugging." format COMMAND_BUILDER)
+
+        // Give developers a nice friendly warning if they've specified task.opts and are using a threaded job.
+        config.getTaskOpts match {
+          case Some(taskOpts) => warn("%s was specified in config, but is not being used because job is being executed with ThreadJob. You probably want to run %s=%s." format (TASK_JVM_OPTS, COMMAND_BUILDER, classOf[ShellCommandBuilder].getName))
+          case _ => None
+        }
+
+        // No command class was specified, so execute the job in this process
+        // using a threaded job.
+        new ThreadJob(SamzaContainer(taskName, partitions, config))
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJob.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJob.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJob.scala
new file mode 100644
index 0000000..a9ecd97
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJob.scala
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.local
+
+import org.apache.samza.job.ApplicationStatus.New
+import org.apache.samza.job.ApplicationStatus.Running
+import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
+import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
+import org.apache.samza.job.ApplicationStatus
+import org.apache.samza.job.StreamJob
+import grizzled.slf4j.Logging
+import java.util.concurrent.CountDownLatch
+import java.io.BufferedReader
+import java.io.InputStreamReader
+import java.io.InputStream
+import java.io.OutputStream
+import org.apache.samza.SamzaException
+
+class ProcessJob(processBuilder: ProcessBuilder) extends StreamJob with Logging {
+  var jobStatus: Option[ApplicationStatus] = None
+  var process: Process = null
+
+  def submit: StreamJob = {
+    val waitForThreadStart = new CountDownLatch(1)
+    jobStatus = Some(New)
+
+    // create a non-daemon thread to make job runner block until the job finishes.
+    // without this, the proc dies when job runner ends.
+    val procThread = new Thread {
+      override def run {
+        process = processBuilder.start
+
+        // pipe all output to this process's streams
+        val outThread = new Thread(new Piper(process.getInputStream, System.out))
+        val errThread = new Thread(new Piper(process.getErrorStream, System.err))
+        outThread.setDaemon(true)
+        errThread.setDaemon(true)
+        outThread.start
+        errThread.start
+        waitForThreadStart.countDown
+        process.waitFor
+      }
+    }
+
+    procThread.start
+    waitForThreadStart.await
+    jobStatus = Some(Running)
+
+    ProcessJob.this
+  }
+
+  def kill: StreamJob = {
+    process.destroy
+    jobStatus = Some(UnsuccessfulFinish); 
+    ProcessJob.this
+  }
+
+  def waitForFinish(timeoutMs: Long) = {
+    val thread = new Thread {
+      setDaemon(true)
+      override def run {
+        try {
+          process.waitFor
+        } catch {
+          case e: InterruptedException => None
+        }
+      }
+    }
+
+    thread.start
+    thread.join(timeoutMs)
+    thread.interrupt
+    jobStatus.getOrElse(null)
+  }
+
+  def waitForStatus(status: ApplicationStatus, timeoutMs: Long) = {
+    val start = System.currentTimeMillis
+
+    while (System.currentTimeMillis - start < timeoutMs && status != jobStatus) {
+      Thread.sleep(500)
+    }
+
+    jobStatus.getOrElse(null)
+  }
+
+  def getStatus = jobStatus.getOrElse(null)
+}
+
+/**
+ * Silly class to forward bytes from one stream to another. Using this to pipe
+ * output from subprocess to this process' stdout/stderr.
+ */
+class Piper(in: InputStream, out: OutputStream) extends Runnable {
+  def run() {
+    try {
+      val b = new Array[Byte](512)
+      var read = 1;
+      while (read > -1) {
+        read = in.read(b, 0, b.length)
+        if (read > -1) {
+          out.write(b, 0, read)
+          out.flush()
+        }
+      }
+    } catch {
+      case e: Exception => throw new SamzaException("Broken pipe", e);
+    } finally {
+      in.close()
+      out.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJob.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJob.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJob.scala
new file mode 100644
index 0000000..62994b0
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/job/local/ThreadJob.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.job.local
+
+import grizzled.slf4j.Logging
+import org.apache.samza.job.StreamJob
+import org.apache.samza.job.ApplicationStatus
+import org.apache.samza.job.ApplicationStatus.New
+import org.apache.samza.job.ApplicationStatus.Running
+import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
+import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
+
+class ThreadJob(runnable: Runnable) extends StreamJob with Logging {
+  @volatile var jobStatus: Option[ApplicationStatus] = None
+  var thread: Thread = null
+
+  def submit: StreamJob = {
+    jobStatus = Some(New)
+
+    // create a non-daemon thread to make job runner block until the job finishes.
+    // without this, the proc dies when job runner ends.
+    thread = new Thread {
+      override def run {
+        try {
+          runnable.run
+          jobStatus = Some(SuccessfulFinish)
+        } catch {
+          case e: Throwable => {
+            error("Failing job with exception.", e)
+            jobStatus = Some(UnsuccessfulFinish)
+            throw e
+          }
+        }
+      }
+    }
+    thread.setName("ThreadJob")
+    thread.start
+    jobStatus = Some(Running)
+
+    ThreadJob.this
+  }
+
+  def kill: StreamJob = {
+    thread.interrupt
+    ThreadJob.this
+  }
+
+  def waitForFinish(timeoutMs: Long) = {
+    thread.join(timeoutMs)
+    jobStatus.getOrElse(null)
+  }
+
+  def waitForStatus(status: ApplicationStatus, timeoutMs: Long) = {
+    val start = System.currentTimeMillis
+
+    while (System.currentTimeMillis - start < timeoutMs && !status.equals(jobStatus.getOrElse(null))) {
+      Thread.sleep(500)
+    }
+
+    jobStatus.getOrElse(null)
+  }
+
+  def getStatus = jobStatus.getOrElse(null)
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/metrics/JmxServer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/JmxServer.scala b/samza-core/src/main/scala/org/apache/samza/metrics/JmxServer.scala
new file mode 100644
index 0000000..eee213d
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/metrics/JmxServer.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics
+
+import grizzled.slf4j.Logging
+import java.rmi.server.RMIServerSocketFactory
+import java.net.{ InetAddress, ServerSocket }
+import java.rmi.registry.LocateRegistry
+import management.ManagementFactory
+import java.util
+import javax.management.remote.{ JMXConnectorServerFactory, JMXServiceURL }
+import org.apache.samza.config.Config
+
+/**
+ * Programmatically start the JMX server and its accompanying RMI server. This is necessary in order to reliably
+ * and - heh - simply request and know a dynamic port such that processes on the same machine do not collide when
+ * opening JMX servers on the same port. Server will start upon instantiation.
+ *
+ * Note: This server starts the JMX server, which runs in a separate thread and must be stopped or it will prevent
+ * the process from ending.
+ *
+ * @param requestedPort Port on which to start JMX server, 0 for ephemeral
+ */
+class JmxServer(requestedPort: Int) extends Logging {
+  def this() = this(0)
+
+  // Instance construction
+  val (jmxServer, url, actualPort) = {
+    // An RMIServerSocketFactory that will tell what port it opened up.  Imagine that.
+    class UpfrontRMIServerSocketFactory extends RMIServerSocketFactory {
+      var lastSS: ServerSocket = null
+      def createServerSocket(port: Int): ServerSocket = {
+        lastSS = new ServerSocket(port)
+        lastSS
+      }
+    }
+
+    // Check if the system property has been set and, if not, set it to what we need. Warn otherwise.
+    def updateSystemProperty(prop: String, value: String) = {
+      val existingProp = System.getProperty(prop)
+      if (existingProp == null) {
+        debug("Setting new system property of %s to %s" format (prop, value))
+        System.setProperty(prop, value)
+      } else {
+        info("Not overriding system property %s as already has value %s" format (prop, existingProp))
+      }
+    }
+
+    if (System.getProperty("com.sun.management.jmxremote") != null) {
+      warn("System property com.sun.management.jmxremote has been specified, starting the JVM's JMX server as well. " +
+        "This behavior is not well defined and our values will collide with any set on command line.")
+    }
+
+    val hostname = InetAddress.getLocalHost.getHostName
+    info("According to InetAddress.getLocalHost.getHostName we are " + hostname)
+    updateSystemProperty("com.sun.management.jmxremote.authenticate", "false")
+    updateSystemProperty("com.sun.management.jmxremote.ssl", "false")
+    updateSystemProperty("java.rmi.server.hostname", hostname)
+
+    val ssFactory = new UpfrontRMIServerSocketFactory
+    LocateRegistry.createRegistry(requestedPort, null, ssFactory)
+    val actualPort = ssFactory.lastSS.getLocalPort
+    val mbs = ManagementFactory.getPlatformMBeanServer
+    val env = new util.HashMap[String, Object]()
+    val url = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://" + hostname + ":" + actualPort + "/jmxrmi")
+    val jmxServer = JMXConnectorServerFactory.newJMXConnectorServer(url, env, mbs)
+
+    (jmxServer, url.toString, actualPort)
+  }
+
+  jmxServer.start
+  info("Started " + toString)
+
+  /**
+   * Get RMI port the JMX server is listening on.
+   * @return RMI port
+   */
+  def getPort = actualPort
+
+  /**
+   * Get Jmx URL for this server
+   * @return Jmx-Style URL string
+   */
+  def getJmxUrl = url
+
+  /**
+   * Stop the JMX server. Must be called at program end or will prevent termination.
+   */
+  def stop = jmxServer.stop
+
+  override def toString = "JmxServer port=%d url=%s" format (getPort, getJmxUrl)
+}
+


[09/15] initial import.

Posted by cr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java
new file mode 100644
index 0000000..a16378f
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.metrics;
+
+public interface ReadableMetricsRegistryListener {
+  void onCounter(String group, Counter counter);
+
+  void onGauge(String group, Gauge<?> gauge);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/serializers/Deserializer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/serializers/Deserializer.java b/samza-api/src/main/java/org/apache/samza/serializers/Deserializer.java
new file mode 100644
index 0000000..c6264b9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/serializers/Deserializer.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers;
+
+public interface Deserializer<T> {
+  T fromBytes(byte[] bytes);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/serializers/Serde.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/serializers/Serde.java b/samza-api/src/main/java/org/apache/samza/serializers/Serde.java
new file mode 100644
index 0000000..fab1055
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/serializers/Serde.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers;
+
+public interface Serde<T> extends Serializer<T>, Deserializer<T> {
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/serializers/SerdeFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/serializers/SerdeFactory.java b/samza-api/src/main/java/org/apache/samza/serializers/SerdeFactory.java
new file mode 100644
index 0000000..a41a922
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/serializers/SerdeFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers;
+
+import org.apache.samza.config.Config;
+
+public interface SerdeFactory<T> {
+  Serde<T> getSerde(String name, Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/serializers/Serializer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/serializers/Serializer.java b/samza-api/src/main/java/org/apache/samza/serializers/Serializer.java
new file mode 100644
index 0000000..1217698
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/serializers/Serializer.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.serializers;
+
+public interface Serializer<T> {
+  byte[] toBytes(T object);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
new file mode 100644
index 0000000..96dec9b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage;
+
+import java.util.Iterator;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+
+/**
+ * A storage engine for managing state maintained by a stream processor.
+ * 
+ * This interface does not specify any query capabilities, which, of course,
+ * would be query engine specific. Instead it just specifies the minimum
+ * functionality required to reload a storage engine from its changelog as well
+ * as basic lifecycle management.
+ */
+public interface StorageEngine {
+
+  // TODO javadocs for StorageEngine.init
+  void restore(Iterator<IncomingMessageEnvelope> envelopes);
+
+  /**
+   * Flush any cached messages
+   */
+  void flush();
+
+  /**
+   * Close the storage engine
+   */
+  void stop();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/storage/StorageEngineFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/storage/StorageEngineFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StorageEngineFactory.java
new file mode 100644
index 0000000..d7a1b90
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/storage/StorageEngineFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.storage;
+
+import java.io.File;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.MessageCollector;
+
+/**
+ * An object provided by the storage engine implementation to create instances
+ * of the given storage engine type.
+ */
+public interface StorageEngineFactory<K, V> {
+
+  // TODO update javadocs for storage engine factory
+
+  /**
+   * Create an instance of the given storage engine
+   * 
+   * @param dir
+   *          The directory
+   * @param config
+   *          The configuration for the task
+   * @param stream
+   *          The stream used to back this storage engine
+   * @param serde
+   *          The serializer to use for serializing keys and values when reading
+   *          or writing to the store store
+   * @return The storage engine instance
+   */
+  public StorageEngine getStorageEngine(
+    String storeName,
+    File storeDir,
+    Serde<K> keySerde,
+    Serde<V> msgSerde,
+    MessageCollector collector,
+    Config config,
+    MetricsRegistry registry,
+    SystemStreamPartition changeLogSystemStreamPartition);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
new file mode 100644
index 0000000..f70dfea
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelope.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+public class IncomingMessageEnvelope {
+  private final SystemStreamPartition systemStreamPartition;
+  private final String offset;
+  private final Object key;
+  private final Object message;
+
+  public IncomingMessageEnvelope(SystemStreamPartition systemStreamPartition, String offset, Object key, Object message) {
+    this.systemStreamPartition = systemStreamPartition;
+    this.offset = offset;
+    this.key = key;
+    this.message = message;
+  }
+
+  public SystemStreamPartition getSystemStreamPartition() {
+    return systemStreamPartition;
+  }
+
+  public String getOffset() {
+    return offset;
+  }
+
+  public Object getKey() {
+    return key;
+  }
+
+  public Object getMessage() {
+    return message;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + ((message == null) ? 0 : message.hashCode());
+    result = prime * result + ((offset == null) ? 0 : offset.hashCode());
+    result = prime * result + ((systemStreamPartition == null) ? 0 : systemStreamPartition.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    IncomingMessageEnvelope other = (IncomingMessageEnvelope) obj;
+    if (key == null) {
+      if (other.key != null)
+        return false;
+    } else if (!key.equals(other.key))
+      return false;
+    if (message == null) {
+      if (other.message != null)
+        return false;
+    } else if (!message.equals(other.message))
+      return false;
+    if (offset == null) {
+      if (other.offset != null)
+        return false;
+    } else if (!offset.equals(other.offset))
+      return false;
+    if (systemStreamPartition == null) {
+      if (other.systemStreamPartition != null)
+        return false;
+    } else if (!systemStreamPartition.equals(other.systemStreamPartition))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "IncomingMessageEnvelope [systemStreamPartition=" + systemStreamPartition + ", offset=" + offset + ", key=" + key + ", message=" + message + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelopePicker.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelopePicker.java b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelopePicker.java
new file mode 100644
index 0000000..c6fd1d8
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/IncomingMessageEnvelopePicker.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+public interface IncomingMessageEnvelopePicker {
+  void update(IncomingMessageEnvelope envelopes);
+
+  IncomingMessageEnvelope pick();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/OutgoingMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/OutgoingMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/system/OutgoingMessageEnvelope.java
new file mode 100644
index 0000000..9cbb254
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/OutgoingMessageEnvelope.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+public class OutgoingMessageEnvelope {
+  private final SystemStream systemStream;
+  private final String keySerializerName;
+  private final String messageSerializerName;
+  private final Object partitionKey;
+  private final Object key;
+  private final Object message;
+
+  public OutgoingMessageEnvelope(SystemStream systemStream, String keySerializerName, String messageSerializerName, Object partitionKey, Object key, Object message) {
+    this.systemStream = systemStream;
+    this.keySerializerName = keySerializerName;
+    this.messageSerializerName = messageSerializerName;
+    this.partitionKey = partitionKey;
+    this.key = key;
+    this.message = message;
+  }
+
+  public OutgoingMessageEnvelope(SystemStream systemStream, Object partitionKey, Object key, Object message) {
+    this(systemStream, null, null, partitionKey, key, message);
+  }
+
+  public OutgoingMessageEnvelope(SystemStream systemStream, Object key, Object message) {
+    this(systemStream, null, null, key, key, message);
+  }
+
+  public OutgoingMessageEnvelope(SystemStream systemStream, Object message) {
+    this(systemStream, null, null, null, null, message);
+  }
+
+  public SystemStream getSystemStream() {
+    return systemStream;
+  }
+
+  public String getKeySerializerName() {
+    return keySerializerName;
+  }
+
+  public String getMessageSerializerName() {
+    return messageSerializerName;
+  }
+
+  public Object getPartitionKey() {
+    return partitionKey;
+  }
+
+  public Object getKey() {
+    return key;
+  }
+
+  public Object getMessage() {
+    return message;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + ((keySerializerName == null) ? 0 : keySerializerName.hashCode());
+    result = prime * result + ((message == null) ? 0 : message.hashCode());
+    result = prime * result + ((messageSerializerName == null) ? 0 : messageSerializerName.hashCode());
+    result = prime * result + ((partitionKey == null) ? 0 : partitionKey.hashCode());
+    result = prime * result + ((systemStream == null) ? 0 : systemStream.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    OutgoingMessageEnvelope other = (OutgoingMessageEnvelope) obj;
+    if (key == null) {
+      if (other.key != null)
+        return false;
+    } else if (!key.equals(other.key))
+      return false;
+    if (keySerializerName == null) {
+      if (other.keySerializerName != null)
+        return false;
+    } else if (!keySerializerName.equals(other.keySerializerName))
+      return false;
+    if (message == null) {
+      if (other.message != null)
+        return false;
+    } else if (!message.equals(other.message))
+      return false;
+    if (messageSerializerName == null) {
+      if (other.messageSerializerName != null)
+        return false;
+    } else if (!messageSerializerName.equals(other.messageSerializerName))
+      return false;
+    if (partitionKey == null) {
+      if (other.partitionKey != null)
+        return false;
+    } else if (!partitionKey.equals(other.partitionKey))
+      return false;
+    if (systemStream == null) {
+      if (other.systemStream != null)
+        return false;
+    } else if (!systemStream.equals(other.systemStream))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "OutgoingMessageEnvelope [systemStream=" + systemStream + ", keySerializerName=" + keySerializerName + ", messageSerializerName=" + messageSerializerName + ", partitionKey=" + partitionKey + ", key=" + key + ", message=" + message + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
new file mode 100644
index 0000000..c902d41
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import java.util.Set;
+
+import org.apache.samza.Partition;
+
+public interface SystemAdmin {
+  Set<Partition> getPartitions(String streamName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java b/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
new file mode 100644
index 0000000..b182f79
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import java.util.List;
+import java.util.Map;
+
+public interface SystemConsumer {
+  public static int BLOCK_ON_OUTSTANDING_MESSAGES = -1;
+
+  void start();
+
+  void stop();
+
+  void register(SystemStreamPartition systemStreamPartition, String lastReadOffset);
+
+  List<IncomingMessageEnvelope> poll(Map<SystemStreamPartition, Integer> systemStreamPartitions, long timeout) throws InterruptedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java b/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java
new file mode 100644
index 0000000..ae33e8e
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.metrics.MetricsRegistry;
+
+public interface SystemFactory {
+  SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry);
+
+  SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry);
+
+  SystemAdmin getAdmin(String systemName, Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemProducer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemProducer.java b/samza-api/src/main/java/org/apache/samza/system/SystemProducer.java
new file mode 100644
index 0000000..9fd43b3
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemProducer.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+public interface SystemProducer {
+  void start();
+
+  void stop();
+
+  void register(String source);
+
+  void send(String source, OutgoingMessageEnvelope envelope);
+
+  void commit(String source);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemStream.java b/samza-api/src/main/java/org/apache/samza/system/SystemStream.java
new file mode 100644
index 0000000..1de9317
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemStream.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+public class SystemStream {
+  protected final String system;
+  protected final String stream;
+
+  public SystemStream(String system, String stream) {
+    this.system = system;
+    this.stream = stream;
+  }
+
+  public SystemStream(SystemStream other) {
+    this(other.getSystem(), other.getStream());
+  }
+
+  public String getSystem() {
+    return system;
+  }
+
+  public String getStream() {
+    return stream;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((stream == null) ? 0 : stream.hashCode());
+    result = prime * result + ((system == null) ? 0 : system.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    SystemStream other = (SystemStream) obj;
+    if (stream == null) {
+      if (other.stream != null)
+        return false;
+    } else if (!stream.equals(other.stream))
+      return false;
+    if (system == null) {
+      if (other.system != null)
+        return false;
+    } else if (!system.equals(other.system))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "SystemStream [system=" + system + ", stream=" + stream + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
new file mode 100644
index 0000000..4f58f32
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartition.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import org.apache.samza.Partition;
+
+public class SystemStreamPartition extends SystemStream {
+  protected final Partition partition;
+
+  public SystemStreamPartition(String system, String stream, Partition partition) {
+    super(system, stream);
+    this.partition = partition;
+  }
+
+  public SystemStreamPartition(SystemStreamPartition other) {
+    this(other.getSystem(), other.getStream(), other.getPartition());
+  }
+
+  public SystemStreamPartition(SystemStream other, Partition partition) {
+    this(other.getSystem(), other.getStream(), partition);
+  }
+
+  public Partition getPartition() {
+    return partition;
+  }
+  
+  public SystemStream getSystemStream() {
+    return new SystemStream(system, stream);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((partition == null) ? 0 : partition.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    SystemStreamPartition other = (SystemStreamPartition) obj;
+    if (partition == null) {
+      if (other.partition != null)
+        return false;
+    } else if (!partition.equals(other.partition))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "SystemStreamPartition [partition=" + partition + ", system=" + system + ", stream=" + stream + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartitionIterator.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartitionIterator.java b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartitionIterator.java
new file mode 100644
index 0000000..62a5eb7
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemStreamPartitionIterator.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import java.util.ArrayDeque;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+
+import org.apache.samza.SamzaException;
+
+public class SystemStreamPartitionIterator implements Iterator<IncomingMessageEnvelope> {
+  private final SystemConsumer systemConsumer;
+  private final Map<SystemStreamPartition, Integer> fetchMap;
+  private Queue<IncomingMessageEnvelope> peeks;
+
+  public SystemStreamPartitionIterator(SystemConsumer systemConsumer, SystemStreamPartition systemStreamPartition) {
+    this(systemConsumer, systemStreamPartition, 1000);
+  }
+
+  public SystemStreamPartitionIterator(SystemConsumer systemConsumer, SystemStreamPartition systemStreamPartition, int fetchSize) {
+    this.systemConsumer = systemConsumer;
+    this.fetchMap = new HashMap<SystemStreamPartition, Integer>();
+    this.fetchMap.put(systemStreamPartition, fetchSize);
+    this.peeks = new ArrayDeque<IncomingMessageEnvelope>();
+  }
+
+  @Override
+  public boolean hasNext() {
+    refresh();
+
+    return peeks.size() > 0;
+  }
+
+  @Override
+  public IncomingMessageEnvelope next() {
+    refresh();
+
+    if (peeks.size() == 0) {
+      throw new NoSuchElementException();
+    }
+
+    return peeks.poll();
+  }
+
+  @Override
+  public void remove() {
+  }
+
+  private void refresh() {
+    if (peeks.size() == 0) {
+      try {
+        List<IncomingMessageEnvelope> envelopes = systemConsumer.poll(fetchMap, SystemConsumer.BLOCK_ON_OUTSTANDING_MESSAGES);
+
+        if (envelopes != null && envelopes.size() > 0) {
+          peeks.addAll(envelopes);
+        }
+      } catch (InterruptedException e) {
+        throw new SamzaException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java b/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
new file mode 100644
index 0000000..a93cca0
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/ClosableTask.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+public interface ClosableTask {
+  void close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/InitableTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/InitableTask.java b/samza-api/src/main/java/org/apache/samza/task/InitableTask.java
new file mode 100644
index 0000000..c7721bb
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/InitableTask.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+
+public interface InitableTask {
+  void init(Config config, TaskContext context) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/MessageCollector.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/MessageCollector.java b/samza-api/src/main/java/org/apache/samza/task/MessageCollector.java
new file mode 100644
index 0000000..1755529
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/MessageCollector.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.system.OutgoingMessageEnvelope;
+
+public interface MessageCollector {
+  void send(OutgoingMessageEnvelope envelope);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/StreamTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/StreamTask.java b/samza-api/src/main/java/org/apache/samza/task/StreamTask.java
new file mode 100644
index 0000000..04b12c2
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/StreamTask.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+
+public interface StreamTask {
+  void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
new file mode 100644
index 0000000..611507e
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.Partition;
+import org.apache.samza.metrics.MetricsRegistry;
+
+public interface TaskContext {
+  MetricsRegistry getMetricsRegistry();
+
+  Partition getPartition();
+
+  Object getStore(String name);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/TaskCoordinator.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskCoordinator.java b/samza-api/src/main/java/org/apache/samza/task/TaskCoordinator.java
new file mode 100644
index 0000000..192b226
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskCoordinator.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+public interface TaskCoordinator {
+  void commit();
+
+  void shutdown();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
new file mode 100644
index 0000000..fa2b52d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.system.IncomingMessageEnvelope;
+
+/**
+ * Used to get before/after notifications before initializing/closing all tasks
+ * in a given container (JVM/process).
+ */
+public interface TaskLifecycleListener {
+  /**
+   * Called before all tasks in TaskRunner are initialized.
+   */
+  void beforeInit(Config config, TaskContext context);
+
+  /**
+   * Called after all tasks in TaskRunner are initialized.
+   */
+  void afterInit(Config config, TaskContext context);
+
+  /**
+   * Called before a message is processed by a task.
+   */
+  void beforeProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
+
+  /**
+   * Called after a message is processed by a task.
+   */
+  void afterProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
+
+  /**
+   * Called before all tasks in TaskRunner are closed.
+   */
+  void beforeClose(Config config, TaskContext context);
+
+  /**
+   * Called after all tasks in TaskRunner are closed.
+   */
+  void afterClose(Config config, TaskContext context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListenerFactory.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListenerFactory.java b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListenerFactory.java
new file mode 100644
index 0000000..31f32bc
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListenerFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+import org.apache.samza.config.Config;
+
+public interface TaskLifecycleListenerFactory {
+  TaskLifecycleListener getLifecyleListener(String name, Config config);
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/task/WindowableTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/WindowableTask.java b/samza-api/src/main/java/org/apache/samza/task/WindowableTask.java
new file mode 100644
index 0000000..2155ed1
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/task/WindowableTask.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.task;
+
+public interface WindowableTask {
+  void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java b/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
new file mode 100644
index 0000000..96f3148
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemStreamPartition;
+
+/**
+ * <p>
+ * BlockingEnvelopeMap is a helper class for SystemConsumer implementations.
+ * Samza's poll() requirements make implementing SystemConsumers somewhat
+ * tricky. BlockingEnvelopeMap is provided to help other developers write
+ * SystemConsumers.
+ * </p>
+ * 
+ * <p>
+ * SystemConsumers that implement BlockingEnvelopeMap need to add messages using
+ * add (or addAll), and update noMoreMessage using setIsAtHead. The
+ * noMoreMessage variable is used to determine whether a SystemStreamPartition
+ * is "caught up" (has read all possible messages from the underlying system).
+ * For example, with a Kafka system, noMoreMessages would be set to true when
+ * the last message offset returned is equal to the offset high watermark for a
+ * given topic/partition.
+ * </p>
+ */
+public abstract class BlockingEnvelopeMap implements SystemConsumer {
+  private final BlockingEnvelopeMapMetrics metrics;
+  private final ConcurrentHashMap<SystemStreamPartition, BlockingQueue<IncomingMessageEnvelope>> bufferedMessages;
+  private final Map<SystemStreamPartition, Boolean> noMoreMessage;
+  private final int queueSize;
+  private final Clock clock;
+
+  public BlockingEnvelopeMap() {
+    this(1000, new NoOpMetricsRegistry());
+  }
+
+  public BlockingEnvelopeMap(Clock clock) {
+    this(1000, new NoOpMetricsRegistry(), clock);
+  }
+
+  public BlockingEnvelopeMap(int queueSize, MetricsRegistry metricsRegistry) {
+    this(queueSize, metricsRegistry, new Clock() {
+      public long currentTimeMillis() {
+        return System.currentTimeMillis();
+      }
+    });
+  }
+
+  public BlockingEnvelopeMap(int queueSize, MetricsRegistry metricsRegistry, Clock clock) {
+    this.metrics = new BlockingEnvelopeMapMetrics(queueSize, metricsRegistry);
+    this.bufferedMessages = new ConcurrentHashMap<SystemStreamPartition, BlockingQueue<IncomingMessageEnvelope>>();
+    this.noMoreMessage = new ConcurrentHashMap<SystemStreamPartition, Boolean>();
+    this.queueSize = queueSize;
+    this.clock = clock;
+  }
+
+  public void register(SystemStreamPartition systemStreamPartition, String lastReadOffset) {
+    metrics.initMetrics(systemStreamPartition);
+    bufferedMessages.putIfAbsent(systemStreamPartition, newBlockingQueue());
+  }
+
+  protected BlockingQueue<IncomingMessageEnvelope> newBlockingQueue() {
+    return new ArrayBlockingQueue<IncomingMessageEnvelope>(queueSize);
+  }
+
+  public List<IncomingMessageEnvelope> poll(Map<SystemStreamPartition, Integer> systemStreamPartitionAndMaxPerStream, long timeout) throws InterruptedException {
+    long stopTime = clock.currentTimeMillis() + timeout;
+    List<IncomingMessageEnvelope> messagesToReturn = new ArrayList<IncomingMessageEnvelope>();
+
+    metrics.incPoll();
+
+    for (Map.Entry<SystemStreamPartition, Integer> systemStreamPartitionAndMaxCount : systemStreamPartitionAndMaxPerStream.entrySet()) {
+      SystemStreamPartition systemStreamPartition = systemStreamPartitionAndMaxCount.getKey();
+      Integer numMessages = systemStreamPartitionAndMaxCount.getValue();
+      BlockingQueue<IncomingMessageEnvelope> queue = bufferedMessages.get(systemStreamPartition);
+      IncomingMessageEnvelope envelope = null;
+      List<IncomingMessageEnvelope> systemStreamPartitionMessages = new ArrayList<IncomingMessageEnvelope>();
+
+      metrics.incPoll(systemStreamPartition);
+
+      // First, drain all messages up to numMessages without blocking.
+      // Stop when we've filled the request (max numMessages), or when
+      // we get a null envelope back.
+      for (int i = 0; i < numMessages && (i == 0 || envelope != null); ++i) {
+        envelope = queue.poll();
+
+        if (envelope != null) {
+          systemStreamPartitionMessages.add(envelope);
+        }
+      }
+
+      metrics.decBufferedMessageCount(systemStreamPartition, systemStreamPartitionMessages.size());
+
+      // Now block if blocking is allowed and we have no messages.
+      if (systemStreamPartitionMessages.size() == 0) {
+        // How long we can legally block (if timeout > 0)
+        long timeRemaining = stopTime - clock.currentTimeMillis();
+
+        if (timeout == SystemConsumer.BLOCK_ON_OUTSTANDING_MESSAGES) {
+          while (systemStreamPartitionMessages.size() < numMessages && !isAtHead(systemStreamPartition)) {
+            metrics.incBlockingPoll(systemStreamPartition);
+            envelope = queue.poll(1000, TimeUnit.MILLISECONDS);
+
+            if (envelope != null) {
+              systemStreamPartitionMessages.add(envelope);
+              metrics.decBufferedMessageCount(systemStreamPartition, 1);
+            }
+          }
+        } else if (timeout > 0 && timeRemaining > 0) {
+          metrics.incBlockingTimeoutPoll(systemStreamPartition);
+          envelope = queue.poll(timeRemaining, TimeUnit.MILLISECONDS);
+
+          if (envelope != null) {
+            systemStreamPartitionMessages.add(envelope);
+            metrics.decBufferedMessageCount(systemStreamPartition, 1);
+          }
+        }
+      }
+
+      messagesToReturn.addAll(systemStreamPartitionMessages);
+    }
+
+    return messagesToReturn;
+  }
+
+  protected void add(SystemStreamPartition systemStreamPartition, IncomingMessageEnvelope envelope) throws InterruptedException {
+    bufferedMessages.get(systemStreamPartition).put(envelope);
+    metrics.incBufferedMessageCount(systemStreamPartition, 1);
+  }
+
+  protected void addAll(SystemStreamPartition systemStreamPartition, List<IncomingMessageEnvelope> envelopes) throws InterruptedException {
+    BlockingQueue<IncomingMessageEnvelope> queue = bufferedMessages.get(systemStreamPartition);
+
+    for (IncomingMessageEnvelope envelope : envelopes) {
+      queue.put(envelope);
+    }
+
+    metrics.incBufferedMessageCount(systemStreamPartition, envelopes.size());
+  }
+
+  protected int getNumMessagesInQueue(SystemStreamPartition systemStreamPartition) {
+    BlockingQueue<IncomingMessageEnvelope> queue = bufferedMessages.get(systemStreamPartition);
+
+    if (queue == null) {
+      return 0;
+    } else {
+      return queue.size();
+    }
+  }
+
+  protected Boolean setIsAtHead(SystemStreamPartition systemStreamPartition, boolean isAtHead) {
+    metrics.setNoMoreMessages(systemStreamPartition, isAtHead);
+    return noMoreMessage.put(systemStreamPartition, isAtHead);
+  }
+
+  protected boolean isAtHead(SystemStreamPartition systemStreamPartition) {
+    Boolean isAtHead = noMoreMessage.get(systemStreamPartition);
+
+    return getNumMessagesInQueue(systemStreamPartition) == 0 && isAtHead != null && isAtHead.equals(true);
+  }
+
+  public static final class BlockingEnvelopeMapMetrics {
+    private static final String GROUP = "samza.consumers";
+
+    private final MetricsRegistry metricsRegistry;
+    private final ConcurrentHashMap<SystemStreamPartition, Counter> bufferedMessageCountMap;
+    private final ConcurrentHashMap<SystemStreamPartition, Gauge<Boolean>> noMoreMessageGaugeMap;
+    private final ConcurrentHashMap<SystemStreamPartition, Counter> pollCountMap;
+    private final ConcurrentHashMap<SystemStreamPartition, Counter> blockingPollCountMap;
+    private final ConcurrentHashMap<SystemStreamPartition, Counter> blockingPollTimeoutCountMap;
+    // TODO use the queueSize gauge
+    private final Gauge<Integer> queueSize;
+    private final Counter pollCount;
+
+    public BlockingEnvelopeMapMetrics(int queueSize, MetricsRegistry metricsRegistry) {
+      this.metricsRegistry = metricsRegistry;
+      this.bufferedMessageCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
+      this.noMoreMessageGaugeMap = new ConcurrentHashMap<SystemStreamPartition, Gauge<Boolean>>();
+      this.pollCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
+      this.blockingPollCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
+      this.blockingPollTimeoutCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
+      this.queueSize = metricsRegistry.<Integer> newGauge(GROUP, "QueueSize", queueSize);
+      this.pollCount = metricsRegistry.newCounter(GROUP, "PollCount");
+    }
+
+    public void initMetrics(SystemStreamPartition systemStreamPartition) {
+      this.bufferedMessageCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(GROUP, "BufferedMessageCount-" + systemStreamPartition));
+      this.noMoreMessageGaugeMap.putIfAbsent(systemStreamPartition, metricsRegistry.<Boolean> newGauge(GROUP, "NoMoreMessages-" + systemStreamPartition, false));
+      this.pollCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(GROUP, "PollCount-" + systemStreamPartition));
+      this.blockingPollCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(GROUP, "BlockingPollCount-" + systemStreamPartition));
+      this.blockingPollTimeoutCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(GROUP, "BlockingPollTimeoutCount-" + systemStreamPartition));
+    }
+
+    public void incBufferedMessageCount(SystemStreamPartition systemStreamPartition, int count) {
+      this.bufferedMessageCountMap.get(systemStreamPartition).inc(count);
+    }
+
+    public void decBufferedMessageCount(SystemStreamPartition systemStreamPartition, int count) {
+      this.bufferedMessageCountMap.get(systemStreamPartition).dec(count);
+    }
+
+    public void setNoMoreMessages(SystemStreamPartition systemStreamPartition, boolean noMoreMessages) {
+      this.noMoreMessageGaugeMap.get(systemStreamPartition).set(noMoreMessages);
+    }
+
+    public void incBlockingPoll(SystemStreamPartition systemStreamPartition) {
+      this.blockingPollCountMap.get(systemStreamPartition).inc();
+    }
+
+    public void incBlockingTimeoutPoll(SystemStreamPartition systemStreamPartition) {
+      this.blockingPollTimeoutCountMap.get(systemStreamPartition).inc();
+    }
+
+    public void incPoll(SystemStreamPartition systemStreamPartition) {
+      this.pollCountMap.get(systemStreamPartition).inc();
+    }
+
+    public void incPoll() {
+      this.pollCount.inc();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/util/Clock.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/Clock.java b/samza-api/src/main/java/org/apache/samza/util/Clock.java
new file mode 100644
index 0000000..e1a77e6
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/util/Clock.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+public interface Clock {
+  long currentTimeMillis();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java
new file mode 100644
index 0000000..c071c47
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+
+public class NoOpMetricsRegistry implements MetricsRegistry {
+  @Override
+  public Counter newCounter(String group, String name) {
+    return new Counter(name);
+  }
+
+  @Override
+  public <T> Gauge<T> newGauge(String group, String name, T value) {
+    return new Gauge<T>(name, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/main/java/org/apache/samza/util/SinglePartitionSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/SinglePartitionSystemAdmin.java b/samza-api/src/main/java/org/apache/samza/util/SinglePartitionSystemAdmin.java
new file mode 100644
index 0000000..e4ed30b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/util/SinglePartitionSystemAdmin.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.samza.Partition;
+import org.apache.samza.system.SystemAdmin;
+
+/**
+ * A simple helper admin class that defines a single partition (partition 0) for
+ * a given system. This class should be used when a system has no concept of
+ * partitioning, since Samza needs at least one partition for an input stream,
+ * in order to read it.
+ */
+public class SinglePartitionSystemAdmin implements SystemAdmin {
+  private static final Set<Partition> ONE_PARTITION = new HashSet<Partition>();
+
+  static {
+    ONE_PARTITION.add(new Partition(0));
+  }
+
+  @Override
+  public Set<Partition> getPartitions(String streamName) {
+    return Collections.unmodifiableSet(ONE_PARTITION);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/test/java/org/apache/samza/system/TestSystemStreamPartitionIterator.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/system/TestSystemStreamPartitionIterator.java b/samza-api/src/test/java/org/apache/samza/system/TestSystemStreamPartitionIterator.java
new file mode 100644
index 0000000..3ecabab
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/system/TestSystemStreamPartitionIterator.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.system;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.junit.Test;
+
+import org.apache.samza.Partition;
+
+public class TestSystemStreamPartitionIterator {
+  private static final SystemStreamPartition SSP = new SystemStreamPartition("test", "test", new Partition(0));
+
+  @Test
+  public void testHasNextShouldWork() {
+    int numMessages = 10;
+    MockSystemConsumer consumer = new MockSystemConsumer(numMessages);
+    SystemStreamPartitionIterator iterator = new SystemStreamPartitionIterator(consumer, SSP);
+
+    while (iterator.hasNext()) {
+      assertEquals(--numMessages, iterator.next().getMessage());
+    }
+
+    assertFalse(iterator.hasNext());
+    assertEquals(0, numMessages);
+  }
+
+  @Test
+  public void testNextWithoutHasNextCallShouldWorkWhenAvailableAndFailWhenNot() {
+    int numMessages = 10;
+    MockSystemConsumer consumer = new MockSystemConsumer(numMessages);
+    SystemStreamPartitionIterator iterator = new SystemStreamPartitionIterator(consumer, SSP);
+
+    for (int i = 0; i < numMessages; ++i) {
+      assertEquals(numMessages - i - 1, iterator.next().getMessage());
+    }
+
+    assertFalse(iterator.hasNext());
+
+    try {
+      iterator.next();
+      fail("Expected not to get any more messages from iterator.");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testNoMessages() {
+    int numMessages = 0;
+    MockSystemConsumer consumer = new MockSystemConsumer(numMessages);
+    SystemStreamPartitionIterator iterator = new SystemStreamPartitionIterator(consumer, SSP);
+
+    assertFalse(iterator.hasNext());
+
+    try {
+      iterator.next();
+      fail("Expected not to get any more messages from iterator.");
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+  }
+
+  public class MockSystemConsumer implements SystemConsumer {
+    private int numPollReturnsWithMessages;
+
+    public MockSystemConsumer(int numMessages) {
+      this.numPollReturnsWithMessages = numMessages;
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void stop() {
+    }
+
+    @Override
+    public void register(SystemStreamPartition systemStreamPartition, String startingOffset) {
+    }
+
+    @Override
+    public List<IncomingMessageEnvelope> poll(Map<SystemStreamPartition, Integer> systemStreamPartitions, long timeout) {
+      List<IncomingMessageEnvelope> list = new ArrayList<IncomingMessageEnvelope>();
+
+      if (numPollReturnsWithMessages-- > 0) {
+        list.add(new IncomingMessageEnvelope(SSP, "", null, numPollReturnsWithMessages));
+      }
+
+      return list;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/test/java/org/apache/samza/util/TestBlockingEnvelopeMap.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/util/TestBlockingEnvelopeMap.java b/samza-api/src/test/java/org/apache/samza/util/TestBlockingEnvelopeMap.java
new file mode 100644
index 0000000..2aed464
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/util/TestBlockingEnvelopeMap.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import static org.junit.Assert.*;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+import org.apache.samza.Partition;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+public class TestBlockingEnvelopeMap {
+  private static final SystemStreamPartition SSP = new SystemStreamPartition("test", "test", new Partition(0));
+  private static final IncomingMessageEnvelope envelope = new IncomingMessageEnvelope(SSP, null, null, null);
+  private static final Map<SystemStreamPartition, Integer> FETCH = new HashMap<SystemStreamPartition, Integer>();
+
+  static {
+    FETCH.put(SSP, 10);
+  }
+
+  @Test
+  public void testEmptyMapReturnsEmptyList() throws InterruptedException {
+    BlockingEnvelopeMap map = new MockBlockingEnvelopeMap();
+    map.register(SSP, "0");
+    map.poll(FETCH, 0);
+    map.poll(FETCH, 30);
+    map.setIsAtHead(SSP, true);
+    map.poll(FETCH, -1);
+  }
+
+  @Test
+  public void testShouldBlockAtLeast100Ms() throws InterruptedException {
+    BlockingEnvelopeMap map = new MockBlockingEnvelopeMap();
+    map.register(SSP, "0");
+    long now = System.currentTimeMillis();
+    map.poll(FETCH, 100);
+    assertTrue(System.currentTimeMillis() - now >= 100);
+  }
+
+  @Test
+  public void testShouldGetSomeMessages() throws InterruptedException {
+    BlockingEnvelopeMap map = new MockBlockingEnvelopeMap();
+    map.register(SSP, "0");
+    map.add(SSP, envelope);
+    List<IncomingMessageEnvelope> envelopes = map.poll(FETCH, 0);
+    assertEquals(1, envelopes.size());
+    map.add(SSP, envelope);
+    map.add(SSP, envelope);
+    envelopes = map.poll(FETCH, 0);
+    assertEquals(2, envelopes.size());
+  }
+
+  @Test
+  public void testShouldNotReturnMoreEnvelopesThanAllowed() throws InterruptedException {
+    BlockingEnvelopeMap map = new MockBlockingEnvelopeMap();
+    int maxMessages = FETCH.get(SSP);
+
+    map.register(SSP, "0");
+
+    for (int i = 0; i < 3 * maxMessages; ++i) {
+      map.add(SSP, envelope);
+    }
+
+    assertEquals(3 * maxMessages, map.getNumMessagesInQueue(SSP));
+    assertEquals(maxMessages, map.poll(FETCH, 0).size());
+    assertEquals(2 * maxMessages, map.getNumMessagesInQueue(SSP));
+    assertEquals(maxMessages, map.poll(FETCH, 30).size());
+    assertEquals(maxMessages, map.getNumMessagesInQueue(SSP));
+    assertEquals(maxMessages, map.poll(FETCH, 0).size());
+    assertEquals(0, map.getNumMessagesInQueue(SSP));
+    assertEquals(0, map.poll(FETCH, 0).size());
+  }
+
+  @Test
+  public void testShouldBlockWhenNotAtHead() throws InterruptedException {
+    MockQueue q = new MockQueue();
+    final BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(q);
+
+    map.register(SSP, "0");
+
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+       // Should trigger a take() call.
+          map.poll(FETCH, -1);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    });
+
+    t.setDaemon(true);
+    t.start();
+    q.awaitPollTimeout();
+    t.join(60000);
+
+    // 1000 = blocking timeout constant
+    assertEquals(1000, q.timeout);
+    assertFalse(t.isAlive());
+  }
+
+  @Test
+  public void testShouldPollWithATimeout() throws InterruptedException {
+    MockQueue q = new MockQueue();
+    final BlockingEnvelopeMap map = new MockBlockingEnvelopeMap(q);
+
+    map.register(SSP, "0");
+
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          // Should trigger a poll(100, TimeUnit.MILLISECONDS) call.
+          map.poll(FETCH, 100);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    });
+
+    t.setDaemon(true);
+    t.start();
+    q.awaitPollTimeout();
+    t.join(60000);
+
+    assertEquals(100, q.timeout);
+    assertFalse(t.isAlive());
+  }
+
+  public class MockQueue extends ArrayBlockingQueue<IncomingMessageEnvelope> {
+    private static final long serialVersionUID = 1L;
+    private final CountDownLatch pollTimeoutBarrier;
+    private long timeout;
+
+    public MockQueue() {
+      super(1000);
+      this.pollTimeoutBarrier = new CountDownLatch(1);
+    }
+
+    public void awaitPollTimeout() throws InterruptedException {
+      pollTimeoutBarrier.await();
+    }
+
+    @Override
+    public IncomingMessageEnvelope poll(long timeout, TimeUnit unit) {
+      this.timeout = timeout;
+
+      pollTimeoutBarrier.countDown();
+
+      return envelope;
+    }
+  }
+
+  public class MockBlockingEnvelopeMap extends BlockingEnvelopeMap {
+    private final BlockingQueue<IncomingMessageEnvelope> injectedQueue;
+
+    public MockBlockingEnvelopeMap() {
+      this(null);
+    }
+
+    public MockBlockingEnvelopeMap(BlockingQueue<IncomingMessageEnvelope> injectedQueue) {
+      super();
+
+      this.injectedQueue = injectedQueue;
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void stop() {
+    }
+
+    protected BlockingQueue<IncomingMessageEnvelope> newBlockingQueue() {
+      if (injectedQueue != null) {
+        return injectedQueue;
+      } else {
+        return super.newBlockingQueue();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/test/java/org/apache/samza/util/TestNoOpMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/util/TestNoOpMetricsRegistry.java b/samza-api/src/test/java/org/apache/samza/util/TestNoOpMetricsRegistry.java
new file mode 100644
index 0000000..78d2824
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/util/TestNoOpMetricsRegistry.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+
+public class TestNoOpMetricsRegistry {
+  @Test
+  public void testNoOpMetricsHappyPath() {
+    NoOpMetricsRegistry registry = new NoOpMetricsRegistry();
+    Counter counter1 = registry.newCounter("testc", "a");
+    Counter counter2 = registry.newCounter("testc", "b");
+    Counter counter3 = registry.newCounter("testc2", "c");
+    Gauge<String> gauge1 = registry.newGauge("testg", "a", "1");
+    Gauge<String> gauge2 = registry.newGauge("testg", "b", "2");
+    Gauge<String> gauge3 = registry.newGauge("testg", "c", "3");
+    Gauge<String> gauge4 = registry.newGauge("testg2", "d", "4");
+    counter1.inc();
+    counter2.inc(2);
+    counter3.inc(4);
+    gauge1.set("5");
+    gauge2.set("6");
+    gauge3.set("7");
+    gauge4.set("8");
+    assertEquals(counter1.getCount(), 1);
+    assertEquals(counter2.getCount(), 2);
+    assertEquals(counter3.getCount(), 4);
+    assertEquals(gauge1.getValue(), "5");
+    assertEquals(gauge2.getValue(), "6");
+    assertEquals(gauge3.getValue(), "7");
+    assertEquals(gauge4.getValue(), "8");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-api/src/test/java/org/apache/samza/util/TestSinglePartitionSystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/util/TestSinglePartitionSystemAdmin.java b/samza-api/src/test/java/org/apache/samza/util/TestSinglePartitionSystemAdmin.java
new file mode 100644
index 0000000..d120818
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/util/TestSinglePartitionSystemAdmin.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.util;
+
+import java.util.Set;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.samza.Partition;
+
+public class TestSinglePartitionSystemAdmin {
+  @Test
+  public void testShouldGetASinglePartition() {
+    SinglePartitionSystemAdmin admin = new SinglePartitionSystemAdmin();
+    Set<Partition> partitions1 = admin.getPartitions("a");
+    Set<Partition> partitions2 = admin.getPartitions("b");
+    assertEquals(partitions1, partitions2);
+    assertEquals(partitions1.size(), 1);
+    assertEquals(partitions1.iterator().next(), new Partition(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/5ff71e51/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala
new file mode 100644
index 0000000..364e489
--- /dev/null
+++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.samza.checkpoint.file
+
+import java.io.File
+import java.io.FileOutputStream
+import scala.collection.JavaConversions._
+import scala.io.Source
+import org.apache.samza.SamzaException
+import org.apache.samza.serializers.CheckpointSerde
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.config.Config
+import org.apache.samza.Partition
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.config.FileSystemCheckpointManagerConfig.Config2FSCP
+import org.apache.samza.checkpoint.CheckpointManagerFactory
+import org.apache.samza.checkpoint.CheckpointManager
+import org.apache.samza.checkpoint.Checkpoint
+import java.io.FileNotFoundException
+
+class FileSystemCheckpointManager(
+  jobName: String,
+  root: File,
+  serde: CheckpointSerde = new CheckpointSerde) extends CheckpointManager {
+
+  def register(partition: Partition) {
+  }
+
+  def writeCheckpoint(partition: Partition, checkpoint: Checkpoint) {
+    val bytes = serde.toBytes(checkpoint)
+    val fos = new FileOutputStream(getFile(jobName, partition))
+
+    fos.write(bytes)
+    fos.close
+  }
+
+  def readLastCheckpoint(partition: Partition): Checkpoint = {
+    try {
+      val bytes = Source.fromFile(getFile(jobName, partition)).map(_.toByte).toArray
+
+      serde.fromBytes(bytes)
+    } catch {
+      case e: FileNotFoundException => null
+    }
+  }
+
+  def start {
+    if (!root.exists) {
+      throw new SamzaException("Root directory for file system checkpoint manager does not exist: %s" format root)
+    }
+  }
+
+  def stop {}
+
+  private def getFile(jobName: String, partition: Partition) =
+    new File(root, "%s-%d" format (jobName, partition.getPartitionId))
+}
+
+class FileSystemCheckpointManagerFactory extends CheckpointManagerFactory {
+  def getCheckpointManager(config: Config, registry: MetricsRegistry) = {
+    val name = config
+      .getName
+      .getOrElse(throw new SamzaException("Missing job name in configs"))
+    val root = config
+      .getFileSystemCheckpointRoot
+      .getOrElse(throw new SamzaException("Missing checkpoint root in configs"))
+    new FileSystemCheckpointManager(name, new File(root))
+  }
+}